diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 000000000..5aa9a33cc --- /dev/null +++ b/.editorconfig @@ -0,0 +1,275 @@ +root = true + +[*] +charset = utf-8 +end_of_line = lf +insert_final_newline = true +max_line_length = 100 +# ij_formatter_off_tag = @formatter:off +# ij_formatter_on_tag = @formatter:on +# ij_formatter_tags_enabled = false +# ij_smart_tabs = false +# ij_wrap_on_typing = false + +[*.java] +indent_size = 4 +indent_style = space +tab_width = 4 +ij_continuation_indent_size = 8 +# ij_java_align_consecutive_assignments = false +# ij_java_align_consecutive_variable_declarations = false +# ij_java_align_group_field_declarations = false +# ij_java_align_multiline_annotation_parameters = false +# ij_java_align_multiline_array_initializer_expression = false +# ij_java_align_multiline_assignment = false +# ij_java_align_multiline_binary_operation = false +# ij_java_align_multiline_chained_methods = false +# ij_java_align_multiline_extends_list = false +# ij_java_align_multiline_for = true +# ij_java_align_multiline_method_parentheses = false +# ij_java_align_multiline_parameters = true +# ij_java_align_multiline_parameters_in_calls = false +# ij_java_align_multiline_parenthesized_expression = false +# ij_java_align_multiline_records = true +# ij_java_align_multiline_resources = true +# ij_java_align_multiline_ternary_operation = false +# ij_java_align_multiline_text_blocks = false +# ij_java_align_multiline_throws_list = false +# ij_java_align_subsequent_simple_methods = false +# ij_java_align_throws_keyword = false +# ij_java_annotation_parameter_wrap = off +# ij_java_array_initializer_new_line_after_left_brace = false +# ij_java_array_initializer_right_brace_on_new_line = false +# ij_java_array_initializer_wrap = off +# ij_java_assert_statement_colon_on_next_line = false +# ij_java_assert_statement_wrap = off +# ij_java_assignment_wrap = off +ij_java_binary_operation_sign_on_next_line = true +ij_java_binary_operation_wrap = normal +# ij_java_blank_lines_after_anonymous_class_header = 0 +# ij_java_blank_lines_after_class_header = 0 +# ij_java_blank_lines_after_imports = 1 +# ij_java_blank_lines_after_package = 1 +# ij_java_blank_lines_around_class = 1 +# ij_java_blank_lines_around_field = 0 +# ij_java_blank_lines_around_field_in_interface = 0 +# ij_java_blank_lines_around_initializer = 1 +# ij_java_blank_lines_around_method = 1 +# ij_java_blank_lines_around_method_in_interface = 1 +# ij_java_blank_lines_before_class_end = 0 +# ij_java_blank_lines_before_imports = 1 +# ij_java_blank_lines_before_method_body = 0 +# ij_java_blank_lines_before_package = 0 +# ij_java_block_brace_style = end_of_line +# ij_java_block_comment_at_first_column = true +ij_java_call_parameters_new_line_after_left_paren = true +# ij_java_call_parameters_right_paren_on_new_line = false +ij_java_call_parameters_wrap = on_every_item +# ij_java_case_statement_on_separate_line = true +# ij_java_catch_on_new_line = false +# ij_java_class_annotation_wrap = split_into_lines +# ij_java_class_brace_style = end_of_line +ij_java_class_count_to_use_import_on_demand = 9999 +# ij_java_class_names_in_javadoc = 1 +# ij_java_do_not_indent_top_level_class_members = false +# ij_java_do_not_wrap_after_single_annotation = false +# ij_java_do_while_brace_force = never +# ij_java_doc_add_blank_line_after_description = true +ij_java_doc_add_blank_line_after_param_comments = true +ij_java_doc_add_blank_line_after_return = true +# ij_java_doc_add_p_tag_on_empty_lines = true +ij_java_doc_align_exception_comments = false +ij_java_doc_align_param_comments = false +ij_java_doc_do_not_wrap_if_one_line = true +ij_java_doc_enable_formatting = true +# ij_java_doc_enable_leading_asterisks = true +ij_java_doc_indent_on_continuation = true +ij_java_doc_keep_empty_lines = true +# ij_java_doc_keep_empty_parameter_tag = true +# ij_java_doc_keep_empty_return_tag = true +# ij_java_doc_keep_empty_throws_tag = true +# ij_java_doc_keep_invalid_tags = true +# ij_java_doc_param_description_on_new_line = false +ij_java_doc_preserve_line_breaks = false +# ij_java_doc_use_throws_not_exception_tag = true +# ij_java_else_on_new_line = false +# ij_java_entity_dd_suffix = EJB +# ij_java_entity_eb_suffix = Bean +# ij_java_entity_hi_suffix = Home +# ij_java_entity_lhi_prefix = Local +# ij_java_entity_lhi_suffix = Home +# ij_java_entity_li_prefix = Local +# ij_java_entity_pk_class = java.lang.String +# ij_java_entity_vo_suffix = VO +# ij_java_enum_constants_wrap = off +# ij_java_extends_keyword_wrap = off +# ij_java_extends_list_wrap = off +# ij_java_field_annotation_wrap = split_into_lines +# ij_java_finally_on_new_line = false +# ij_java_for_brace_force = never +# ij_java_for_statement_new_line_after_left_paren = false +# ij_java_for_statement_right_paren_on_new_line = false +# ij_java_for_statement_wrap = off +# ij_java_generate_final_locals = false +# ij_java_generate_final_parameters = false +# ij_java_if_brace_force = never +ij_java_imports_layout = org.apache.flink.**,|,org.apache.flink.shaded.**,|,*,|,javax.**,|,java.**,|,scala.**,|,$* +# ij_java_indent_case_from_switch = true +# ij_java_insert_inner_class_imports = false +# ij_java_insert_override_annotation = true +# ij_java_keep_blank_lines_before_right_brace = 2 +# ij_java_keep_blank_lines_between_package_declaration_and_header = 2 +# ij_java_keep_blank_lines_in_code = 2 +# ij_java_keep_blank_lines_in_declarations = 2 +# ij_java_keep_control_statement_in_one_line = true +# ij_java_keep_first_column_comment = true +# ij_java_keep_indents_on_empty_lines = false +# ij_java_keep_line_breaks = true +# ij_java_keep_multiple_expressions_in_one_line = false +# ij_java_keep_simple_blocks_in_one_line = false +# ij_java_keep_simple_classes_in_one_line = false +# ij_java_keep_simple_lambdas_in_one_line = false +# ij_java_keep_simple_methods_in_one_line = false +# ij_java_label_indent_absolute = false +# ij_java_label_indent_size = 0 +# ij_java_lambda_brace_style = end_of_line +ij_java_layout_static_imports_separately = true +# ij_java_line_comment_add_space = false +# ij_java_line_comment_at_first_column = true +# ij_java_message_dd_suffix = EJB +# ij_java_message_eb_suffix = Bean +# ij_java_method_annotation_wrap = split_into_lines +# ij_java_method_brace_style = end_of_line +ij_java_method_call_chain_wrap = on_every_item +ij_java_method_parameters_new_line_after_left_paren = true +# ij_java_method_parameters_right_paren_on_new_line = false +ij_java_method_parameters_wrap = on_every_item +# ij_java_modifier_list_wrap = false +ij_java_names_count_to_use_import_on_demand = 9999 +# ij_java_new_line_after_lparen_in_record_header = false +# ij_java_packages_to_use_import_on_demand = java.awt.*,javax.swing.* +# ij_java_parameter_annotation_wrap = off +# ij_java_parentheses_expression_new_line_after_left_paren = false +# ij_java_parentheses_expression_right_paren_on_new_line = false +# ij_java_place_assignment_sign_on_next_line = false +# ij_java_prefer_longer_names = true +# ij_java_prefer_parameters_wrap = false +# ij_java_record_components_wrap = normal +# ij_java_repeat_synchronized = true +# ij_java_replace_instanceof_and_cast = false +# ij_java_replace_null_check = true +# ij_java_replace_sum_lambda_with_method_ref = true +# ij_java_resource_list_new_line_after_left_paren = false +# ij_java_resource_list_right_paren_on_new_line = false +# ij_java_resource_list_wrap = off +# ij_java_rparen_on_new_line_in_record_header = false +# ij_java_session_dd_suffix = EJB +# ij_java_session_eb_suffix = Bean +# ij_java_session_hi_suffix = Home +# ij_java_session_lhi_prefix = Local +# ij_java_session_lhi_suffix = Home +# ij_java_session_li_prefix = Local +# ij_java_session_si_suffix = Service +# ij_java_space_after_closing_angle_bracket_in_type_argument = false +# ij_java_space_after_colon = true +# ij_java_space_after_comma = true +# ij_java_space_after_comma_in_type_arguments = true +# ij_java_space_after_for_semicolon = true +# ij_java_space_after_quest = true +# ij_java_space_after_type_cast = true +# ij_java_space_before_annotation_array_initializer_left_brace = false +# ij_java_space_before_annotation_parameter_list = false +# ij_java_space_before_array_initializer_left_brace = false +# ij_java_space_before_catch_keyword = true +# ij_java_space_before_catch_left_brace = true +# ij_java_space_before_catch_parentheses = true +# ij_java_space_before_class_left_brace = true +# ij_java_space_before_colon = true +# ij_java_space_before_colon_in_foreach = true +# ij_java_space_before_comma = false +# ij_java_space_before_do_left_brace = true +# ij_java_space_before_else_keyword = true +# ij_java_space_before_else_left_brace = true +# ij_java_space_before_finally_keyword = true +# ij_java_space_before_finally_left_brace = true +# ij_java_space_before_for_left_brace = true +# ij_java_space_before_for_parentheses = true +# ij_java_space_before_for_semicolon = false +# ij_java_space_before_if_left_brace = true +# ij_java_space_before_if_parentheses = true +# ij_java_space_before_method_call_parentheses = false +# ij_java_space_before_method_left_brace = true +# ij_java_space_before_method_parentheses = false +# ij_java_space_before_opening_angle_bracket_in_type_parameter = false +# ij_java_space_before_quest = true +# ij_java_space_before_switch_left_brace = true +# ij_java_space_before_switch_parentheses = true +# ij_java_space_before_synchronized_left_brace = true +# ij_java_space_before_synchronized_parentheses = true +# ij_java_space_before_try_left_brace = true +# ij_java_space_before_try_parentheses = true +# ij_java_space_before_type_parameter_list = false +# ij_java_space_before_while_keyword = true +# ij_java_space_before_while_left_brace = true +# ij_java_space_before_while_parentheses = true +# ij_java_space_inside_one_line_enum_braces = false +# ij_java_space_within_empty_array_initializer_braces = false +# ij_java_space_within_empty_method_call_parentheses = false +# ij_java_space_within_empty_method_parentheses = false +# ij_java_spaces_around_additive_operators = true +# ij_java_spaces_around_assignment_operators = true +# ij_java_spaces_around_bitwise_operators = true +# ij_java_spaces_around_equality_operators = true +# ij_java_spaces_around_lambda_arrow = true +# ij_java_spaces_around_logical_operators = true +# ij_java_spaces_around_method_ref_dbl_colon = false +# ij_java_spaces_around_multiplicative_operators = true +# ij_java_spaces_around_relational_operators = true +# ij_java_spaces_around_shift_operators = true +# ij_java_spaces_around_type_bounds_in_type_parameters = true +# ij_java_spaces_around_unary_operator = false +# ij_java_spaces_within_angle_brackets = false +# ij_java_spaces_within_annotation_parentheses = false +# ij_java_spaces_within_array_initializer_braces = false +# ij_java_spaces_within_braces = false +# ij_java_spaces_within_brackets = false +# ij_java_spaces_within_cast_parentheses = false +# ij_java_spaces_within_catch_parentheses = false +# ij_java_spaces_within_for_parentheses = false +# ij_java_spaces_within_if_parentheses = false +# ij_java_spaces_within_method_call_parentheses = false +# ij_java_spaces_within_method_parentheses = false +# ij_java_spaces_within_parentheses = false +# ij_java_spaces_within_switch_parentheses = false +# ij_java_spaces_within_synchronized_parentheses = false +# ij_java_spaces_within_try_parentheses = false +# ij_java_spaces_within_while_parentheses = false +# ij_java_special_else_if_treatment = true +# ij_java_subclass_name_suffix = Impl +# ij_java_ternary_operation_signs_on_next_line = false +# ij_java_ternary_operation_wrap = off +# ij_java_test_name_suffix = Test +# ij_java_throws_keyword_wrap = off +# ij_java_throws_list_wrap = off +# ij_java_use_external_annotations = false +# ij_java_use_fq_class_names = false +# ij_java_use_relative_indents = false +# ij_java_use_single_class_imports = true +ij_java_variable_annotation_wrap = normal +# ij_java_visibility = public +# ij_java_while_brace_force = never +# ij_java_while_on_new_line = false +# ij_java_wrap_comments = false +ij_java_wrap_first_method_in_call_chain = true +# ij_java_wrap_long_lines = false + +[*.out] +insert_final_newline = false + +[*.xml] +indent_style = tab +indent_size = 4 + +[*.py] +indent_style = space +indent_size = 4 diff --git a/.github/workflows/push_pr.yml b/.github/workflows/push_pr.yml index 3e9e1c205..69854c344 100644 --- a/.github/workflows/push_pr.yml +++ b/.github/workflows/push_pr.yml @@ -16,13 +16,28 @@ # limitations under the License. ################################################################################ -name: CI +# We need to specify repo related information here since Apache INFRA doesn't differentiate +# between several workflows with the same names while preparing a report for GHA usage +# https://infra-reports.apache.org/#ghactions +name: Flink Connector Kafka CI on: [push, pull_request] concurrency: group: ${{ github.workflow }}-${{ github.ref }} cancel-in-progress: true jobs: compile_and_test: + strategy: + matrix: + flink: [ 1.20.0 ] + jdk: [ '8, 11, 17, 21' ] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: - flink_version: 1.17-SNAPSHOT + flink_version: ${{ matrix.flink }} + jdk_version: ${{ matrix.jdk }} + python_test: + strategy: + matrix: + flink: [ 1.20.0 ] + uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils + with: + flink_version: ${{ matrix.flink }} diff --git a/.github/workflows/weekly.yml b/.github/workflows/weekly.yml index 9d34bc1b4..3d7341cc4 100644 --- a/.github/workflows/weekly.yml +++ b/.github/workflows/weekly.yml @@ -16,7 +16,10 @@ # limitations under the License. ################################################################################ -name: Nightly +# We need to specify repo related information here since Apache INFRA doesn't differentiate +# between several workflows with the same names while preparing a report for GHA usage +# https://infra-reports.apache.org/#ghactions +name: Weekly Flink Connector Kafka CI on: schedule: - cron: "0 0 * * 0" @@ -26,8 +29,22 @@ jobs: if: github.repository_owner == 'apache' strategy: matrix: - flink: [1.16-SNAPSHOT, 1.17-SNAPSHOT] + flink_branches: [{ + flink: 1.20-SNAPSHOT, + branch: main + }, { + flink: 1.19.1, + branch: v3.3 + }, { + flink: 1.20.0, + branch: v3.3 + }, { + flink: 1.20.0, + branch: v3.4 + }] uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils with: - flink_version: ${{ matrix.flink }} + flink_version: ${{ matrix.flink_branches.flink }} + connector_branch: ${{ matrix.flink_branches.branch }} + jdk_version: ${{ matrix.flink_branches.jdk || '8, 11, 17, 21' }} run_dependency_convergence: false diff --git a/.gitignore b/.gitignore index 5f0068cda..485c27aa6 100644 --- a/.gitignore +++ b/.gitignore @@ -35,4 +35,21 @@ out/ tools/flink tools/flink-* tools/releasing/release -tools/japicmp-output \ No newline at end of file +tools/japicmp-output + +# Generated file, do not store in git +flink-connector-kafka/.idea +flink-python/pyflink/datastream/connectors/kafka_connector_version.py +flink-python/apache_flink_connectors_kafka.egg-info/ +flink-python/.tox/ +flink-python/build +flink-python/dist +flink-python/dev/download +flink-python/dev/.conda/ +flink-python/dev/log/ +flink-python/dev/.stage.txt +flink-python/dev/install_command.sh +flink-python/dev/lint-python.sh +flink-python/dev/build-wheels.sh +flink-python/dev/glibc_version_fix.h +flink-python/dev/dev-requirements.txt diff --git a/.idea/vcs.xml b/.idea/vcs.xml index a5e4331ed..264965d21 100644 --- a/.idea/vcs.xml +++ b/.idea/vcs.xml @@ -20,5 +20,6 @@ + \ No newline at end of file diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties new file mode 100644 index 000000000..405de879d --- /dev/null +++ b/.mvn/wrapper/maven-wrapper.properties @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. + +wrapperVersion=3.3.2 + +# updating the Maven version requires updates to certain documentation and verification logic +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.6/apache-maven-3.8.6-bin.zip +distributionSha256Sum=ccf20a80e75a17ffc34d47c5c95c98c39d426ca17d670f09cd91e877072a9309 +wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.3.2/maven-wrapper-3.3.2.jar +wrapperSha256Sum=3d8f20ce6103913be8b52aef6d994e0c54705fb527324ceb9b835b338739c7a8 diff --git a/NOTICE b/NOTICE index 02d9298c8..a0ed01945 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Flink Kafka Connector -Copyright 2014-2022 The Apache Software Foundation +Copyright 2014-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md new file mode 100644 index 000000000..abe7a7532 --- /dev/null +++ b/docs/content.zh/docs/connectors/datastream/dynamic-kafka.md @@ -0,0 +1,250 @@ +--- +title: Dynamic Kafka +weight: 3 +type: docs +aliases: +- /dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from Kafka topics from one or more Kafka clusters. +The Dynamic Kafka connector discovers the clusters and topics using a Kafka metadata service and can achieve reading in a dynamic fashion, facilitating changes in +topics and/or clusters, without requiring a job restart. This is especially useful when you need to read a new Kafka cluster/topic and/or stop reading +an existing Kafka cluster/topic (cluster migration/failover/other infrastructure changes) and when you need direct integration with Hybrid Source. The solution +makes these operations automated so that they are transparent to Kafka consumers. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka kafka >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +### Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "DynamicKafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setProperties(properties) + .build(); + + env.fromSource(source, WatermarkStrategy.noWatermarks(), "Dynamic Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} +The following properties are **required** for building a DynamicKafkaSource: + +The Kafka metadata service, configured by setKafkaMetadataService(KafkaMetadataService) +The stream ids to subscribe, see the following Kafka stream subscription section for more details. +Deserializer to parse Kafka messages, see the [Kafka Source Documentation]({{< ref "docs/connectors/datastream/kafka" >}}#deserializer) for more details. + +### Kafka Stream Subscription +The Dynamic Kafka Source provides 2 ways of subscribing to Kafka stream(s). +* A set of Kafka stream ids. For example: + {{< tabs "DynamicKafkaSource#setStreamIds" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamIds(Set.of("stream-a", "stream-b")); + ``` + {{< /tab >}} + {{< /tabs >}} +* A regex pattern that subscribes to all Kafka stream ids that match the provided regex. For example: + {{< tabs "DynamicKafkaSource#setStreamPattern" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamPattern(Pattern.of("stream.*")); + ``` + {{< /tab >}} + {{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Properties +There are configuration options in DynamicKafkaSourceOptions that can be configured in the properties through the builder: + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
stream-metadata-discovery-interval-ms
required-1LongThe interval in milliseconds for the source to discover the changes in stream metadata. A non-positive value disables the stream metadata discovery.
stream-metadata-discovery-failure-threshold
required1IntegerThe number of consecutive failures before letting the exception from Kafka metadata service discovery trigger jobmanager failure and global failover. The default is one to at least catch startup failures.
+ + +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#additional-properties) for +a list of applicable properties. + +### Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatorcurrentEmitEventTimeLagn/aThe time span from the record event timestamp to the time the record is emitted by the source connector¹: currentEmitEventTimeLag = EmitTime - EventTime.Gauge
watermarkLagn/aThe time span that the watermark lags behind the wall clock time: watermarkLag = CurrentTime - WatermarkGauge
sourceIdleTimen/aThe time span that the source has not processed any record: sourceIdleTime = CurrentTime - LastRecordProcessTimeGauge
pendingRecordsn/aThe number of records that have not been fetched by the source. e.g. the available records after the consumer offset in a Kafka partition.Gauge
kafkaClustersCountn/aThe total number of Kafka clusters read by this reader.Gauge
+ +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#monitoring) for +the KafkaSourceReader metrics that are also reported. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the [Kafka Source documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-source). This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from one or more clusters. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from one or more clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. + +{{< top >}} diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 039bdeca8..4a90ece31 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -36,7 +36,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client 当前 Kafka client 向后兼容 0.10.0 或更高版本的 Kafka broker。 有关 Kafka 兼容性的更多细节,请参考 [Kafka 官方文档](https://kafka.apache.org/protocol.html#protocol_compatibility)。 -{{< connector_artifact flink-connector-kafka 3.0.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} 如果使用 Kafka source,```flink-connector-base``` 也需要包含在依赖中: @@ -45,7 +45,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client Flink 目前的流连接器还不是二进制发行版的一部分。 [在此处]({{< ref "docs/dev/configuration/overview" >}})可以了解到如何链接它们,从而在集群中运行。 -{{< py_download_link "kafka" >}} +{{< py_connector_download_link "kafka" >}} ## Kafka Source {{< hint info >}} @@ -222,14 +222,12 @@ Kafka Source 支持流式和批式两种运行模式。默认情况下,KafkaSo Kafka consumer 的配置可以参考 [Apache Kafka 文档](http://kafka.apache.org/documentation/#consumerconfigs)。 请注意,即使指定了以下配置项,构建器也会将其覆盖: -- ```key.deserializer``` 始终设置为 ByteArrayDeserializer -- ```value.deserializer``` 始终设置为 ByteArrayDeserializer - ```auto.offset.reset.strategy``` 被 OffsetsInitializer#getAutoOffsetResetStrategy() 覆盖 - ```partition.discovery.interval.ms``` 会在批模式下被覆盖为 -1 ### 动态分区检查 为了在不重启 Flink 作业的情况下处理 Topic 扩容或新建 Topic 等场景,可以将 Kafka Source 配置为在提供的 Topic / Partition -订阅模式下定期检查新分区。要启用动态分区检查,请将 ```partition.discovery.interval.ms``` 设置为非负值: +订阅模式下定期检查新分区。要启用动态分区检查,请将 ```partition.discovery.interval.ms``` 设置为正值: {{< tabs "KafkaSource#PartitionDiscovery" >}} @@ -248,7 +246,7 @@ KafkaSource.builder() \ {{< /tabs >}} {{< hint warning >}} -分区检查功能默认**不开启**。需要显式地设置分区检查间隔才能启用此功能。 +分区检查间隔默认为5分钟。需要显式地设置分区检查间隔为非正数才能关闭此功能。 {{< /hint >}} ### 事件时间和水印 diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 408cb1a2f..9df680df8 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -48,7 +48,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', 'topic' = 'user_behavior', @@ -81,7 +81,7 @@ CREATE TABLE KafkaTable ( topic STRING NOT NULL Kafka 记录的 Topic 名。 - R + R/W partition @@ -127,7 +127,7 @@ CREATE TABLE KafkaTable ( ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'timestamp', + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `partition` BIGINT METADATA VIRTUAL, `offset` BIGINT METADATA VIRTUAL, `user_id` BIGINT, @@ -151,7 +151,7 @@ CREATE TABLE KafkaTable ( ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector @@ -191,17 +191,17 @@ CREATE TABLE KafkaTable (
topic
- required for sink + 可选 (无) String - 当表用作 source 时读取数据的 topic 名。亦支持用分号间隔的 topic 列表,如 'topic-1;topic-2'。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。当表被用作 sink 时,该配置表示写入的 topic 名。注意 sink 表不支持 topic 列表。 + 当表用作 source 时读取数据的 topic 名,或当表用作 sink 时写入的 topic 名。它还支持通过分号分隔的 topic 列表,如 'topic-1;topic-2' 来作为 source 的 topic 列表。注意,“topic-pattern”和“topic”只能指定其中一个。对于 sink 来说,topic 名是写入数据的 topic。它还支持 sink 的 topic 列表。提供的 topic 列表被视为 `topic` 元数据列的有效值的允许列表。如果提供了列表,对于 sink 表,“topic”元数据列是可写的并且必须指定。
topic-pattern
可选 (无) String - 匹配读取 topic 名称的正则表达式。在作业开始运行时,所有匹配该正则表达式的 topic 都将被 Kafka consumer 订阅。注意,对 source 表而言,'topic' 和 'topic-pattern' 两个选项只能使用其中一个。 + 用于读取或写入的 topic 名称模式的正则表达式。所有匹配指定正则表达式的 topic 名称将在作业开始运行时被消费者订阅。对于 sink 来说,`topic` 元数据列是可写的,必须提供并且与 `topic-pattern` 正则表达式匹配。注意,“topic-pattern”和“topic”只能指定其中一个。
properties.bootstrap.servers
@@ -338,9 +338,17 @@ CREATE TABLE KafkaTable (
scan.topic-partition-discovery.interval
可选 - (无) + 5分钟 Duration - Consumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。 + Consumer 定期探测动态创建的 Kafka topic 和 partition 的时间间隔。需要显式地设置'scan.topic-partition-discovery.interval'为0才能关闭此功能 + + +
scan.parallelism
+ optional + no + (none) + Integer + 定义 Kafka source 算子的并行度。默认情况下会使用全局默认并行度。
sink.partitioner
@@ -389,7 +397,7 @@ Kafka 消息的消息键和消息体部分都可以使用某种 [格式]({{< ref ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -415,7 +423,7 @@ ROW<`user_id` BIGINT, `item_id` BIGINT, `behavior` STRING> ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -572,29 +580,29 @@ Source 输出的 watermark 由读取的分区中最小的 watermark 决定。 请参阅 [Kafka watermark 策略]({{< ref "docs/dev/datastream/event-time/generating_watermarks" >}}#watermark-策略和-kafka-连接器) 以获取更多细节。 ### 安全 -要启用加密和认证相关的安全配置,只需将安全配置加上 "properties." 前缀配置在 Kafka 表上即可。下面的代码片段展示了如何配置 Kafka 表以使用 -PLAIN 作为 SASL 机制并提供 JAAS 配置: +要启用加密和认证相关的安全配置,只需将安全配置加上 "properties." 前缀配置在 Kafka 表上即可。下面的代码片段展示了当依赖 SQL client JAR 时, 如何配置 Kafka 表 +以使用 PLAIN 作为 SASL 机制并提供 JAAS 配置: ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... 'properties.security.protocol' = 'SASL_PLAINTEXT', 'properties.sasl.mechanism' = 'PLAIN', - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' + 'properties.sasl.jaas.config' = 'org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' ) ``` -另一个更复杂的例子,使用 SASL_SSL 作为安全协议并使用 SCRAM-SHA-256 作为 SASL 机制: +另一个更复杂的例子,当依赖 SQL client JAR 时,使用 SASL_SSL 作为安全协议并使用 SCRAM-SHA-256 作为 SASL 机制: ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... @@ -610,13 +618,13 @@ CREATE TABLE KafkaTable ( /* 将 SASL 机制配置为 as SCRAM-SHA-256 */ 'properties.sasl.mechanism' = 'SCRAM-SHA-256', /* 配置 JAAS */ - 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' + 'properties.sasl.jaas.config' = 'org.apache.flink.kafka.shaded.org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";' ) ``` -如果在作业 JAR 中 Kafka 客户端依赖的类路径被重置了(relocate class),登录模块(login module)的类路径可能会不同,因此请根据登录模块在 -JAR 中实际的类路径来改写以上配置。例如在 SQL client JAR 中,Kafka client 依赖被重置在了 `org.apache.flink.kafka.shaded.org.apache.kafka` 路径下, -因此 plain 登录模块的类路径应写为 `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule`。 +在作业 JAR 中 Kafka 客户端依赖的类路径被重置了(relocate class),登录模块(login module)的类路径可能会不同,因此需要根据登录模块在 +JAR 中实际的类路径来改写以上配置。在 SQL client JAR 中,Kafka client 依赖被重置在了 `org.apache.flink.kafka.shaded.org.apache.kafka` +路径下,因此以上的代码片段中 plain 登录模块的类路径写为 `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule`。 关于安全配置的详细描述,请参阅 Apache Kafka 文档中的"安全"一节。 diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 40df1fa20..bacaae52b 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -38,7 +38,7 @@ Upsert Kafka 连接器支持以 upsert 方式从 Kafka topic 中读取数据并 依赖 ------------ -{{< sql_download_table "upsert-kafka" >}} +{{< sql_connector_download_table "kafka" >}} Upsert Kafka 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Upsert Kafka 连接器。 @@ -119,7 +119,7 @@ of all available metadata fields. 必选 (none) String - 用于读取和写入的 Kafka topic 名称。 + 当表用作 source 时读取数据的 topic 名,或当表用作 sink 时写入的 topic 名。它还支持通过分号分隔的 topic 列表,如 'topic-1;topic-2' 来作为 source 的 topic 列表。注意,“topic-pattern”和“topic”只能指定其中一个。对于 sink 来说,topic 名是写入数据的 topic。它还支持 sink 的 topic 列表。提供的 topic 列表被视为 `topic` 元数据列的有效值的允许列表。如果提供了列表,对于 sink 表,“topic”元数据列是可写的并且必须指定。
properties.bootstrap.servers
@@ -136,7 +136,7 @@ of all available metadata fields. 该选项可以传递任意的 Kafka 参数。选项的后缀名必须匹配定义在 Kafka 参数文档中的参数名。 Flink 会自动移除 选项名中的 "properties." 前缀,并将转换后的键名以及值传入 KafkaClient。 例如,你可以通过 'properties.allow.auto.create.topics' = 'false' - 来禁止自动创建 topic。 但是,某些选项,例如'key.deserializer''value.deserializer' 是不允许通过该方式传递参数,因为 Flink 会重写这些参数的值。 + 来禁止自动创建 topic。 但是,某些选项,例如'auto.offset.reset' 是不允许通过该方式传递参数,因为 Flink 会重写这些参数的值。 @@ -180,6 +180,14 @@ of all available metadata fields. + +
scan.parallelism
+ optional + no + (none) + Integer + 定义 upsert-kafka source 算子的并行度。默认情况下会使用全局默认并行度。 +
sink.parallelism
可选 @@ -221,7 +229,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, diff --git a/docs/content/docs/connectors/datastream/dynamic-kafka.md b/docs/content/docs/connectors/datastream/dynamic-kafka.md new file mode 100644 index 000000000..e64b93e67 --- /dev/null +++ b/docs/content/docs/connectors/datastream/dynamic-kafka.md @@ -0,0 +1,250 @@ +--- +title: Dynamic Kafka +weight: 3 +type: docs +aliases: + - /dev/connectors/dynamic-kafka.html +--- + + +# Dynamic Kafka Source _`Experimental`_ + +Flink provides an [Apache Kafka](https://kafka.apache.org) connector for reading data from Kafka topics from one or more Kafka clusters. +The Dynamic Kafka connector discovers the clusters and topics using a Kafka metadata service and can achieve reading in a dynamic fashion, facilitating changes in +topics and/or clusters, without requiring a job restart. This is especially useful when you need to read a new Kafka cluster/topic and/or stop reading +an existing Kafka cluster/topic (cluster migration/failover/other infrastructure changes) and when you need direct integration with Hybrid Source. The solution +makes these operations automated so that they are transparent to Kafka consumers. + +## Dependency + +For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). + +{{< connector_artifact flink-connector-kafka kafka >}} + +Flink's streaming connectors are not part of the binary distribution. +See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). + +## Dynamic Kafka Source +{{< hint info >}} +This part describes the Dynamic Kafka Source based on the new +[data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. +{{< /hint >}} + +### Usage + +Dynamic Kafka Source provides a builder class to initialize the DynamicKafkaSource. The code snippet +below shows how to build a DynamicKafkaSource to consume messages from the earliest offset of the +stream "input-stream" and deserialize only the value of the +ConsumerRecord as a string, using "MyKafkaMetadataService" to resolve the cluster(s) and topic(s) +corresponding to "input-stream". + +{{< tabs "DynamicKafkaSource" >}} +{{< tab "Java" >}} +```java + +DynamicKafkaSource source = DynamicKafkaSource.builder() + .setKafkaMetadataService(new MyKafkaMetadataService()) + .setStreamIds(Collections.singleton("input-stream")) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) + .setProperties(properties) + .build(); + +env.fromSource(source, WatermarkStrategy.noWatermarks(), "Dynamic Kafka Source"); +``` +{{< /tab >}} +{{< /tabs >}} +The following properties are **required** for building a DynamicKafkaSource: + +The Kafka metadata service, configured by setKafkaMetadataService(KafkaMetadataService) +The stream ids to subscribe, see the following Kafka stream subscription section for more details. +Deserializer to parse Kafka messages, see the [Kafka Source Documentation]({{< ref "docs/connectors/datastream/kafka" >}}#deserializer) for more details. + +### Kafka Stream Subscription +The Dynamic Kafka Source provides 2 ways of subscribing to Kafka stream(s). +* A set of Kafka stream ids. For example: + {{< tabs "DynamicKafkaSource#setStreamIds" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamIds(Set.of("stream-a", "stream-b")); + ``` + {{< /tab >}} + {{< /tabs >}} +* A regex pattern that subscribes to all Kafka stream ids that match the provided regex. For example: + {{< tabs "DynamicKafkaSource#setStreamPattern" >}} + {{< tab "Java" >}} + ```java + DynamicKafkaSource.builder().setStreamPattern(Pattern.of("stream.*")); + ``` + {{< /tab >}} + {{< /tabs >}} + +### Kafka Metadata Service + +An interface is provided to resolve the logical Kafka stream(s) into the corresponding physical +topic(s) and cluster(s). Typically, these implementations are based on services that align well +with internal Kafka infrastructure--if that is not available, an in-memory implementation +would also work. An example of in-memory implementation can be found in our tests. + +This source achieves its dynamic characteristic by periodically polling this Kafka metadata service +for any changes to the Kafka stream(s) and reconciling the reader tasks to subscribe to the new +Kafka metadata returned by the service. For example, in the case of a Kafka migration, the source would +swap from one cluster to the new cluster when the service makes that change in the Kafka stream metadata. + +### Additional Properties +There are configuration options in DynamicKafkaSourceOptions that can be configured in the properties through the builder: + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionRequiredDefaultTypeDescription
stream-metadata-discovery-interval-ms
required-1LongThe interval in milliseconds for the source to discover the changes in stream metadata. A non-positive value disables the stream metadata discovery.
stream-metadata-discovery-failure-threshold
required1IntegerThe number of consecutive failures before letting the exception from Kafka metadata service discovery trigger jobmanager failure and global failover. The default is one to at least catch startup failures.
+ + +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#additional-properties) for +a list of applicable properties. + +### Metrics + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatorcurrentEmitEventTimeLagn/aThe time span from the record event timestamp to the time the record is emitted by the source connector¹: currentEmitEventTimeLag = EmitTime - EventTime.Gauge
watermarkLagn/aThe time span that the watermark lags behind the wall clock time: watermarkLag = CurrentTime - WatermarkGauge
sourceIdleTimen/aThe time span that the source has not processed any record: sourceIdleTime = CurrentTime - LastRecordProcessTimeGauge
pendingRecordsn/aThe number of records that have not been fetched by the source. e.g. the available records after the consumer offset in a Kafka partition.Gauge
kafkaClustersCountn/aThe total number of Kafka clusters read by this reader.Gauge
+ +In addition to this list, see the [regular Kafka connector]({{< ref "docs/connectors/datastream/kafka" >}}#monitoring) for +the KafkaSourceReader metrics that are also reported. + +### Additional Details + +For additional details on deserialization, event time and watermarks, idleness, consumer offset +committing, security, and more, you can refer to the [Kafka Source documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-source). This is possible because the +Dynamic Kafka Source leverages components of the Kafka Source, and the implementation will be +discussed in the next section. + +### Behind the Scene +{{< hint info >}} +If you are interested in how Kafka source works under the design of new data source API, you may +want to read this part as a reference. For details about the new data source API, +[documentation of data source]({{< ref "docs/dev/datastream/sources.md" >}}) and +FLIP-27 +provide more descriptive discussions. +{{< /hint >}} + + +Under the abstraction of the new data source API, Dynamic Kafka Source consists of the following components: +#### Source Split +A source split in Dynamic Kafka Source represents a partition of a Kafka topic, with cluster information. It +consists of: +* A Kafka cluster id that can be resolved by the Kafka metadata service. +* A Kafka Source Split (TopicPartition, starting offset, stopping offset). + +You can check the class `DynamicKafkaSourceSplit` for more details. + +#### Split Enumerator + +This enumerator is responsible for discovering and assigning splits from one or more clusters. At startup, the +enumerator will discover metadata belonging to the Kafka stream ids. Using the metadata, it can +initialize KafkaSourceEnumerators to handle the functions of assigning splits to the readers. In addition, +source events will be sent to the source reader to reconcile the metadata. This enumerator has the ability to poll the +KafkaMetadataService, periodically for stream discovery. In addition, restarting enumerators when metadata changes involve +clearing outdated metrics since clusters may be removed and so should their metrics. + +#### Source Reader + +This reader is responsible for reading from one or more clusters and using the KafkaSourceReader to fetch +records from topics and clusters based on the metadata. When new metadata is discovered by the enumerator, +the reader will reconcile metadata changes to possibly restart the KafkaSourceReader to read from the new +set of topics and clusters. + +#### Kafka Metadata Service + +This interface represents the source of truth for the current metadata for the configured Kafka stream ids. +Metadata that is removed in between polls is considered non-active (e.g. removing a cluster from the +return value, means that a cluster is non-active and should not be read from). The cluster metadata +contains an immutable Kafka cluster id, the set of topics, and properties needed to connect to the +Kafka cluster. + +#### FLIP 246 + +To understand more behind the scenes, please read [FLIP-246](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=217389320) +for more details and discussion. + +{{< top >}} diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 7589b8d3c..bddcefec9 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -36,12 +36,12 @@ The version of the client it uses may change between Flink releases. Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. For details on Kafka compatibility, please refer to the official [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). -{{< connector_artifact flink-connector-kafka 3.0.0 >}} +{{< connector_artifact flink-connector-kafka kafka >}} Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). -{{< py_download_link "kafka" >}} +{{< py_connector_download_link "kafka" >}} ## Kafka Source {{< hint info >}} @@ -235,8 +235,6 @@ for more details. Please note that the following keys will be overridden by the builder even if it is configured: -- ```key.deserializer``` is always set to ```ByteArrayDeserializer``` -- ```value.deserializer``` is always set to ```ByteArrayDeserializer``` - ```auto.offset.reset.strategy``` is overridden by ```OffsetsInitializer#getAutoOffsetResetStrategy()``` for the starting offsets - ```partition.discovery.interval.ms``` is overridden to -1 when @@ -245,7 +243,7 @@ it is configured: ### Dynamic Partition Discovery In order to handle scenarios like topic scaling-out or topic creation without restarting the Flink job, Kafka source can be configured to periodically discover new partitions under provided -topic-partition subscribing pattern. To enable partition discovery, set a non-negative value for +topic-partition subscribing pattern. To enable partition discovery, set a positive value for property ```partition.discovery.interval.ms```: {{< tabs "KafkaSource#PartitionDiscovery" >}} @@ -264,8 +262,7 @@ KafkaSource.builder() \ {{< /tabs >}} {{< hint warning >}} -Partition discovery is **disabled** by default. You need to explicitly set the partition discovery -interval to enable this feature. +The partition discovery interval is 5 minutes by default. To **disable** this feature, you need to explicitly set the partition discovery interval to a non-positive value. {{< /hint >}} ### Event Time and Watermarks @@ -466,6 +463,25 @@ client dependencies in the job JAR, so you may need to rewrite it with the actua For detailed explanations of security configurations, please refer to the "Security" section in Apache Kafka documentation. +## Kafka Rack Awareness + +Kafka rack awareness allows Flink to select and control the cloud region and availability zone that Kafka consumers read from, based on the Rack ID. This feature reduces network costs and latency since it allows consumers to connect to the closest Kafka brokers, possibly colocated in the same cloud region and availability zone. +A client's rack is indicated using the `client.rack` config, and should correspond to a broker's `broker.rack` config. + +https://kafka.apache.org/documentation/#consumerconfigs_client.rack + +### RackId + +setRackIdSupplier() is the Builder method allows us to determine the consumer's rack. If provided, the Supplier will be run when the consumer is set up on the Task Manager, and the consumer's `client.rack` configuration will be set to the value. + +One of the ways this can be implemented is by making setRackId equal to an environment variable within your taskManager, for instance: + +``` +.setRackIdSupplier(() -> System.getenv("TM_NODE_AZ")) +``` + +The "TM_NODE_AZ" is the name of the environment variable in the TaskManager container that contains the zone we want to use. + ### Behind the Scene {{< hint info >}} If you are interested in how Kafka source works under the design of new data source API, you may diff --git a/docs/content/docs/connectors/table/kafka.md b/docs/content/docs/connectors/table/kafka.md index 3c9e739c2..12b0821c3 100644 --- a/docs/content/docs/connectors/table/kafka.md +++ b/docs/content/docs/connectors/table/kafka.md @@ -35,7 +35,7 @@ The Kafka connector allows for reading data from and writing data into Kafka top Dependencies ------------ -{{< sql_download_table "kafka" >}} +{{< sql_connector_download_table "kafka" >}} The Kafka connector is not part of the binary distribution. See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). @@ -50,7 +50,7 @@ CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', 'topic' = 'user_behavior', @@ -83,7 +83,7 @@ Read-only columns must be declared `VIRTUAL` to exclude them during an `INSERT I topic STRING NOT NULL Topic name of the Kafka record. - R + R/W partition @@ -129,7 +129,7 @@ The extended `CREATE TABLE` example demonstrates the syntax for exposing these m ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'timestamp', + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `partition` BIGINT METADATA VIRTUAL, `offset` BIGINT METADATA VIRTUAL, `user_id` BIGINT, @@ -154,7 +154,7 @@ The following example shows how to access both Kafka and Debezium metadata field ```sql CREATE TABLE KafkaTable ( - `event_time` TIMESTAMP(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format + `event_time` TIMESTAMP_LTZ(3) METADATA FROM 'value.source.timestamp' VIRTUAL, -- from Debezium format `origin_table` STRING METADATA FROM 'value.source.table' VIRTUAL, -- from Debezium format `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector @@ -196,11 +196,11 @@ Connector Options
topic
- required for sink + optional yes (none) String - Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified for sources. When the table is used as sink, the topic name is the topic to write data to. Note topic list is not supported for sinks. + Topic name(s) to read data from when the table is used as source, or topics for writing when the table is used as sink. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified. For sinks, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified.
topic-pattern
@@ -208,7 +208,7 @@ Connector Options yes (none) String - The regular expression for a pattern of topic names to read from. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. Note, only one of "topic-pattern" and "topic" can be specified for sources. + The regular expression for a pattern of topic names to read from or write to. All topics with names that match the specified regular expression will be subscribed by the consumer when the job starts running. For sinks, the `topic` metadata column is writable, must be provided and match the `topic-pattern` regex. Note, only one of "topic-pattern" and "topic" can be specified.
properties.bootstrap.servers
@@ -233,7 +233,7 @@ Connector Options (none) String - This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. + This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'auto.offset.reset'. @@ -337,6 +337,7 @@ Connector Options
scan.bounded.mode
optional + no unbounded Enum Bounded mode for Kafka consumer, valid values are 'latest-offset', 'group-offsets', 'timestamp' and 'specific-offsets'. @@ -364,9 +365,17 @@ Connector Options
scan.topic-partition-discovery.interval
optional yes - (none) + 5 minutes Duration - Interval for consumer to discover dynamically created Kafka topics and partitions periodically. + Interval for consumer to discover dynamically created Kafka topics and partitions periodically. To disable this feature, you need to explicitly set the 'scan.topic-partition-discovery.interval' value to 0. + + +
scan.parallelism
+ optional + no + (none) + Integer + Defines the parallelism of the Kafka source operator. If not set, the global default parallelism is used.
sink.partitioner
@@ -435,7 +444,7 @@ options are prefixed with the format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -461,7 +470,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING @@ -608,8 +617,8 @@ Besides enabling Flink's checkpointing, you can also choose three different mode * `none`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. * `at-least-once` (default setting): This guarantees that no records will be lost (although they can be duplicated). * `exactly-once`: Kafka transactions will be used to provide exactly-once semantic. Whenever you write - to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_committed` - or `read_uncommitted` - the latter one is the default value) for any application consuming records + to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_uncommitted` + or `read_committed` - the latter one is the default value) for any application consuming records from Kafka. Please refer to [Kafka documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-and-fault-tolerance) for more caveats about delivery guarantees. @@ -629,13 +638,13 @@ for more details. ### Security In order to enable security configurations including encryption and authentication, you just need to setup security configurations with "properties." prefix in table options. The code snippet below shows configuring Kafka table to -use PLAIN as SASL mechanism and provide JAAS configuration: +use PLAIN as SASL mechanism and provide JAAS configuration when using SQL client JAR : ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... @@ -644,13 +653,13 @@ CREATE TABLE KafkaTable ( 'properties.sasl.jaas.config' = 'org.apache.kafka.common.security.plain.PlainLoginModule required username=\"username\" password=\"password\";' ) ``` -For a more complex example, use SASL_SSL as the security protocol and use SCRAM-SHA-256 as SASL mechanism: +For a more complex example, use SASL_SSL as the security protocol and use SCRAM-SHA-256 as SASL mechanism when using SQL client JAR : ```sql CREATE TABLE KafkaTable ( `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, - `ts` TIMESTAMP(3) METADATA FROM 'timestamp' + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp' ) WITH ( 'connector' = 'kafka', ... @@ -672,8 +681,9 @@ CREATE TABLE KafkaTable ( Please note that the class path of the login module in `sasl.jaas.config` might be different if you relocate Kafka client dependencies, so you may need to rewrite it with the actual class path of the module in the JAR. -For example if you are using SQL client JAR, which has relocate Kafka client dependencies to `org.apache.flink.kafka.shaded.org.apache.kafka`, -the path of plain login module should be `org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule` instead. +SQL client JAR has relocated Kafka client dependencies to `org.apache.flink.kafka.shaded.org.apache.kafka`, +then the path of plain login module in code snippets above need to be +`org.apache.flink.kafka.shaded.org.apache.kafka.common.security.plain.PlainLoginModule` when using SQL client JAR. For detailed explanations of security configurations, please refer to the "Security" section in Apache Kafka documentation. diff --git a/docs/content/docs/connectors/table/upsert-kafka.md b/docs/content/docs/connectors/table/upsert-kafka.md index 12a23c5c2..db75309a2 100644 --- a/docs/content/docs/connectors/table/upsert-kafka.md +++ b/docs/content/docs/connectors/table/upsert-kafka.md @@ -47,7 +47,7 @@ key will fall into the same partition. Dependencies ------------ -{{< sql_download_table "upsert-kafka" >}} +{{< sql_connector_download_table "kafka" >}} The Upsert Kafka connector is not part of the binary distribution. See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). @@ -129,7 +129,7 @@ Connector Options required (none) String - The Kafka topic name to read from and write to. + Topic name(s) to read data from when the table is used as source, or topics for writing when the table is used as sink. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of "topic-pattern" and "topic" can be specified. For sinks, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified.
properties.bootstrap.servers
@@ -144,7 +144,7 @@ Connector Options (none) String - This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'key.deserializer' and 'value.deserializer'. + This can set and pass arbitrary Kafka configurations. Suffix names must match the configuration key defined in Kafka Configuration documentation. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying KafkaClient. For example, you can disable automatic topic creation via 'properties.allow.auto.create.topics' = 'false'. But there are some configurations that do not support to set, because Flink will override them, e.g. 'auto.offset.reset'. @@ -192,6 +192,14 @@ Connector Options format which means that key columns appear in the data type for both the key and value format. + +
scan.parallelism
+ optional + no + (none) + Integer + Defines the parallelism of the upsert-kafka source operator. If not set, the global default parallelism is used. +
sink.parallelism
optional @@ -221,6 +229,22 @@ Connector Options By default, this is disabled. Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' must be set to be greater than zero to enable sink buffer flushing. + +
sink.delivery-guarantee
+ optional + no + at-least-once + String + Defines the delivery semantic for the upsert-kafka sink. Valid enumerationns are 'at-least-once', 'exactly-once' and 'none'. See Consistency guarantees for more details. + + +
sink.transactional-id-prefix
+ optional + yes + (none) + String + If the delivery guarantee is configured as 'exactly-once' this value must be set and is used a prefix for the identifier of all opened Kafka transactions. + @@ -240,7 +264,7 @@ prefixed with either the `'key'` or `'value'` plus format identifier. {{< tab "SQL" >}} ```sql CREATE TABLE KafkaTable ( - `ts` TIMESTAMP(3) METADATA FROM 'timestamp', + `ts` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp', `user_id` BIGINT, `item_id` BIGINT, `behavior` STRING, @@ -277,6 +301,19 @@ connector is working in the upsert mode, the last record on the same key will ta reading back as a source. Therefore, the upsert-kafka connector achieves idempotent writes just like the [HBase sink]({{< ref "docs/connectors/table/hbase" >}}). +With Flink's checkpointing enabled, the `upsert-kafka` connector can provide exactly-once delivery guarantees. + +Besides enabling Flink's checkpointing, you can also choose three different modes of operating chosen by passing appropriate `sink.delivery-guarantee` option: + +* `none`: Flink will not guarantee anything. Produced records can be lost or they can be duplicated. +* `at-least-once` (default setting): This guarantees that no records will be lost (although they can be duplicated). +* `exactly-once`: Kafka transactions will be used to provide exactly-once semantic. Whenever you write + to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_uncommitted` + or `read_committed` - the latter one is the default value) for any application consuming records + from Kafka. + +Please refer to [Kafka connector documentation]({{< ref "docs/connectors/datastream/kafka" >}}#kafka-producers-and-fault-tolerance) for more caveats about delivery guarantees. + ### Source Per-Partition Watermarks Flink supports to emit per-partition watermarks for Upsert Kafka. Watermarks are generated inside the Kafka diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml index 4cb2c19ad..e414db962 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 @@ -37,10 +37,6 @@ under the License. - - 7.2.2 - - org.apache.flink @@ -71,6 +67,12 @@ under the License. + + org.apache.flink + flink-connector-base + ${flink.version} + test + org.apache.kafka @@ -110,6 +112,12 @@ under the License. kafka-avro-serializer ${confluent.version} test + + + com.google.guava + guava + + @@ -122,6 +130,10 @@ under the License. org.apache.kafka kafka-clients + + com.google.guava + guava + @@ -159,7 +171,7 @@ under the License. com.google.guava guava - 30.1.1-jre + ${guava.version} @@ -238,7 +250,7 @@ under the License. com.google.guava guava - 30.1.1-jre + ${guava.version} guava.jar jar ${project.build.directory}/dependencies diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java index b22e8a382..e18c035b0 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSinkE2ECase.java @@ -19,6 +19,7 @@ package org.apache.flink.tests.util.kafka; import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; import org.apache.flink.connector.testframe.junit.annotations.TestContext; @@ -28,7 +29,6 @@ import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.test.resources.ResourceTestUtils; -import org.apache.flink.util.DockerImageVersions; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.utility.DockerImageName; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java index 4a036df20..1a2ac1f24 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/KafkaSourceE2ECase.java @@ -18,6 +18,7 @@ package org.apache.flink.tests.util.kafka; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContextFactory; import org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; @@ -28,7 +29,6 @@ import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.test.resources.ResourceTestUtils; -import org.apache.flink.util.DockerImageVersions; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.utility.DockerImageName; diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java index a18976b18..e3b18194a 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientSchemaRegistryITCase.java @@ -19,12 +19,13 @@ package org.apache.flink.tests.util.kafka; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; +import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.connector.testframe.container.FlinkContainers; import org.apache.flink.connector.testframe.container.TestcontainersSettings; import org.apache.flink.test.resources.ResourceTestUtils; import org.apache.flink.test.util.SQLJobSubmission; import org.apache.flink.tests.util.kafka.containers.SchemaRegistryContainer; -import org.apache.flink.tests.util.kafka.test.DockerImageVersions; import io.confluent.kafka.schemaregistry.avro.AvroSchema; import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; @@ -40,11 +41,8 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; import java.nio.file.Path; @@ -60,9 +58,6 @@ /** End-to-end test for SQL client using Avro Confluent Registry format. */ public class SQLClientSchemaRegistryITCase { - private static final Logger LOG = LoggerFactory.getLogger(SQLClientSchemaRegistryITCase.class); - private static final Slf4jLogConsumer LOG_CONSUMER = new Slf4jLogConsumer(LOG); - public static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; public static final String INTER_CONTAINER_REGISTRY_ALIAS = "registry"; private static final Path sqlAvroJar = ResourceTestUtils.getResource(".*avro.jar"); @@ -78,10 +73,9 @@ public class SQLClientSchemaRegistryITCase { @ClassRule public static final KafkaContainer KAFKA = - new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) + KafkaUtil.createKafkaContainer(SQLClientSchemaRegistryITCase.class) .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) - .withLogConsumer(LOG_CONSUMER); + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); @ClassRule public static final SchemaRegistryContainer REGISTRY = @@ -92,7 +86,11 @@ public class SQLClientSchemaRegistryITCase { .dependsOn(KAFKA); public final TestcontainersSettings testcontainersSettings = - TestcontainersSettings.builder().network(NETWORK).logger(LOG).dependsOn(KAFKA).build(); + TestcontainersSettings.builder() + .network(NETWORK) + .logger(KafkaUtil.getLogger("flink", SQLClientSchemaRegistryITCase.class)) + .dependsOn(KAFKA) + .build(); public final FlinkContainers flink = FlinkContainers.builder().withTestcontainersSettings(testcontainersSettings).build(); diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java index 726eceea9..a8c416b0b 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SmokeKafkaITCase.java @@ -29,8 +29,6 @@ import org.apache.flink.test.util.JobSubmission; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; @@ -47,8 +45,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; import org.testcontainers.junit.jupiter.Container; @@ -56,6 +52,7 @@ import java.nio.ByteBuffer; import java.nio.file.Path; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -64,7 +61,6 @@ import java.util.stream.Collectors; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; /** smoke test for the kafka connectors. */ @@ -72,20 +68,22 @@ @Testcontainers class SmokeKafkaITCase { - private static final Logger LOG = LoggerFactory.getLogger(SmokeKafkaITCase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final Network NETWORK = Network.newNetwork(); private static final String EXAMPLE_JAR_MATCHER = "flink-streaming-kafka-test.*"; @Container public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(SmokeKafkaITCase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); public static final TestcontainersSettings TESTCONTAINERS_SETTINGS = - TestcontainersSettings.builder().logger(LOG).dependsOn(KAFKA_CONTAINER).build(); + TestcontainersSettings.builder() + .logger(KafkaUtil.getLogger("flink", SmokeKafkaITCase.class)) + .dependsOn(KAFKA_CONTAINER) + .build(); @RegisterExtension public static final FlinkContainers FLINK = @@ -103,6 +101,10 @@ private static Configuration getConfiguration() { final Configuration flinkConfig = new Configuration(); flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3); flinkConfig.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + // Workaround for FLINK-36454 ; default config is entirely overwritten + flinkConfig.setString( + "env.java.opts.all", + "--add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED"); return flinkConfig; } @@ -137,7 +139,7 @@ public void testKafka() throws Exception { // create the required topics final short replicationFactor = 1; admin.createTopics( - Lists.newArrayList( + Arrays.asList( new NewTopic(inputTopic, 1, replicationFactor), new NewTopic(outputTopic, 1, replicationFactor))) .all() diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties index 358fd81ef..9c49ae58a 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j2-test.properties @@ -32,3 +32,27 @@ appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n #logger.yarn.name = org.testcontainers.shaded.com.github.dockerjava.core #logger.yarn.level = WARN #logger.yarn.appenderRef.console.ref = TestLogger + +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.kafkacontainer.name = container.kafka +logger.kafkacontainer.level = OFF + +logger.flinkcontainer.name = container.flink +logger.flinkcontainer.level = OFF + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = OFF +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml index d4b7dfbcd..974e9d421 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java index 2d05380b8..a6fa83e61 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java @@ -27,7 +27,7 @@ public class KafkaExampleUtil { public static StreamExecutionEnvironment prepareExecutionEnv(ParameterTool parameterTool) throws Exception { - if (parameterTool.getNumberOfParameters() < 5) { + if (parameterTool.getNumberOfParameters() < 4) { System.out.println( "Missing parameters!\n" + "Usage: Kafka --input-topic --output-topic " diff --git a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml index f2d9a2cc1..87498bea7 100644 --- a/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-connector-kafka-e2e-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-connector-kafka-e2e-tests - 4.0-SNAPSHOT + 3.4-SNAPSHOT 4.0.0 diff --git a/flink-connector-kafka-e2e-tests/pom.xml b/flink-connector-kafka-e2e-tests/pom.xml index ddc83d60d..12c881197 100644 --- a/flink-connector-kafka-e2e-tests/pom.xml +++ b/flink-connector-kafka-e2e-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 3.4-SNAPSHOT pom diff --git a/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 b/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 index c7abba76e..109ecf836 100644 --- a/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 +++ b/flink-connector-kafka/archunit-violations/27a0a5e4-29c2-4069-b381-952746c90862 @@ -1 +1 @@ -Method calls method in (FlinkKafkaProducer.java:1327) \ No newline at end of file +Method calls method in (FlinkKafkaProducer.java:1320) diff --git a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a index c1e656185..8d8514add 100644 --- a/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a +++ b/flink-connector-kafka/archunit-violations/86dfd459-67a9-4b26-9b5c-0b0bbf22681a @@ -1,78 +1,90 @@ -org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ +org.apache.flink.connector.kafka.sink.ExactlyOnceKafkaWriterITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.kafka.sink.KafkaSinkITCase does not satisfy: only one of the following predicates match:\ +org.apache.flink.connector.kafka.sink.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.sink.KafkaTransactionLogITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.KafkaWriterFaultToleranceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.sink.KafkaWriterITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.kafka.sink.internal.ProducerPoolImplITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.connector.kafka.source.KafkaSourceLegacyITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.FlinkKafkaInternalProducerITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.KafkaITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.KafkaProducerAtLeastOnceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.KafkaProducerExactlyOnceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleExactlyOnceITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule org.apache.flink.streaming.connectors.kafka.shuffle.KafkaShuffleITCase does not satisfy: only one of the following predicates match:\ * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ -* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension or are , and of type MiniClusterTestEnvironment and annotated with @TestEnv\ * reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ * reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ - or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e new file mode 100644 index 000000000..c27f7c3d4 --- /dev/null +++ b/flink-connector-kafka/archunit-violations/984f05c0-ec82-405e-9bcc-d202dbe7202e @@ -0,0 +1,354 @@ +Class extends class in (YamlFileMetadataService.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class extends class in (StreamKafkaShuffleSink.java:0) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:308) +Constructor (java.util.function.Function, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.kafka.sink.HeaderProvider)> calls method in (KafkaRecordSerializationSchemaBuilder.java:309) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:51) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:53) +Constructor (java.lang.Class, boolean, java.util.Map, java.util.function.Function)> calls method in (KafkaSerializerWrapper.java:54) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:134) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:135) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:136) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:137) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:138) +Constructor (org.apache.flink.connector.base.DeliveryGuarantee, java.util.Properties, java.lang.String, org.apache.flink.api.connector.sink2.Sink$InitContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, java.util.Collection)> calls method in (KafkaWriter.java:173) +Constructor (java.lang.String)> calls method in (KafkaWriterState.java:28) +Constructor (int, int, java.util.function.Function, java.util.function.Consumer)> calls method in (TransactionAborter.java:60) +Constructor (java.lang.Class)> calls constructor ()> in (YamlFileMetadataService.java:270) +Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) +Constructor (java.lang.String, org.apache.flink.connector.kafka.source.enumerator.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext)> calls constructor (java.lang.String)> in (StoppableKafkaEnumContextProxy.java:90) +Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:111) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:253) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, long, boolean)> calls method in (FlinkKafkaConsumerBase.java:251) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1733) +Constructor (java.util.Set)> calls method in (FlinkKafkaProducer.java:1599) +Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1879) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1630) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls constructor ()> in (FlinkKafkaProducerBase.java:120) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:144) +Constructor (java.lang.String, org.apache.flink.streaming.util.serialization.KeyedSerializationSchema, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner)> calls method in (FlinkKafkaProducerBase.java:146) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:584) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> calls method in (AbstractFetcher.java:586) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls constructor (org.apache.flink.api.common.eventtime.WatermarkOutput)> in (AbstractFetcher.java:154) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:152) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:156) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> calls method in (AbstractFetcher.java:159) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int)> calls method in (AbstractPartitionDiscoverer.java:81) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaConsumerThread, org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback)> calls method in (KafkaConsumerThread.java:539) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:136) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:137) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:138) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:139) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:140) +Constructor (org.slf4j.Logger, org.apache.flink.streaming.connectors.kafka.internals.Handover, java.util.Properties, org.apache.flink.streaming.connectors.kafka.internals.ClosableBlockingQueue, java.lang.String, long, boolean, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup)> calls method in (KafkaConsumerThread.java:142) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has generic parameter type >> with type argument depending on in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean)> has parameter of type in (KafkaFetcher.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor, int, int, java.util.Properties)> calls method in (KafkaPartitionDiscoverer.java:50) +Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has generic parameter type >> with type argument depending on in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.streaming.api.functions.source.SourceFunction$SourceContext, java.util.Map, org.apache.flink.util.SerializedValue, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long, java.lang.ClassLoader, java.lang.String, org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema, java.util.Properties, long, org.apache.flink.metrics.MetricGroup, org.apache.flink.metrics.MetricGroup, boolean, org.apache.flink.api.common.typeutils.TypeSerializer, int)> has parameter of type in (KafkaShuffleFetcher.java:0) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:45) +Constructor (java.util.List, java.util.regex.Pattern)> calls method in (KafkaTopicsDescriptor.java:51) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:56) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:57) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:58) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:59) +Constructor (java.lang.String, int, int, int, int)> calls method in (TransactionalIdsGenerator.java:61) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:56) +Constructor (java.lang.String, org.apache.flink.api.common.serialization.TypeInformationSerializationSchema, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties)> calls method in (FlinkKafkaShuffleConsumer.java:59) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:71) +Constructor (java.lang.String, org.apache.flink.api.common.typeutils.TypeSerializer, java.util.Properties, org.apache.flink.api.java.functions.KeySelector, org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$Semantic, int)> calls method in (FlinkKafkaShuffleProducer.java:74) +Constructor (org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffleProducer)> calls constructor (org.apache.flink.streaming.api.functions.sink.SinkFunction)> in (StreamKafkaShuffleSink.java:35) +Constructor (int, org.apache.flink.api.common.serialization.DeserializationSchema, [I, org.apache.flink.api.common.serialization.DeserializationSchema, [I, boolean, [Lorg.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema$MetadataConverter;, org.apache.flink.api.common.typeinfo.TypeInformation, boolean)> calls method in (DynamicKafkaDeserializationSchema.java:70) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:59) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:63) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> calls method in (DynamicKafkaRecordSerializationSchema.java:66) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:175) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:156) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:158) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:161) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:162) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:163) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:169) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:170) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.EncodingFormat, org.apache.flink.table.connector.format.EncodingFormat, [I, [I, java.lang.String, java.lang.String, java.util.Properties, org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner, org.apache.flink.connector.base.DeliveryGuarantee, boolean, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, java.lang.Integer, java.lang.String)> calls method in (KafkaDynamicSink.java:173) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:210) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:194) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:198) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:201) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:203) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:216) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:218) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:220) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:224) +Constructor (org.apache.flink.table.types.DataType, org.apache.flink.table.connector.format.DecodingFormat, org.apache.flink.table.connector.format.DecodingFormat, [I, [I, java.lang.String, java.util.List, java.util.regex.Pattern, java.util.Properties, org.apache.flink.streaming.connectors.kafka.config.StartupMode, java.util.Map, long, org.apache.flink.streaming.connectors.kafka.config.BoundedMode, java.util.Map, long, boolean, java.lang.String)> calls method in (KafkaDynamicSource.java:226) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:70) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:71) +Constructor (org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink$PrecommittingStatefulSinkWriter, org.apache.flink.table.types.DataType, [I, org.apache.flink.streaming.connectors.kafka.table.SinkBufferFlushMode, org.apache.flink.api.common.operators.ProcessingTimeService, java.util.function.Function)> calls method in (ReducingUpsertWriter.java:72) +Field has type in (YamlFileMetadataService.java:0) +Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) +Field has type in (DynamicKafkaSourceReader.java:0) +Field has type in (FlinkKafkaConsumerBase.java:0) +Field has type in (FlinkKafkaProducerBase.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (KafkaShuffleFetcher.java:0) +Field has type in (FlinkKafkaShuffleProducer.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:100) +Method calls method in (FlinkKafkaInternalProducer.java:109) +Method calls method in (FlinkKafkaInternalProducer.java:296) +Method calls method in (FlinkKafkaInternalProducer.java:297) +Method calls method in (FlinkKafkaInternalProducer.java:174) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:254) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:255) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:269) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:265) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:204) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:140) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:98) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:110) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:109) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:125) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:123) +Method calls method in (KafkaRecordSerializationSchemaBuilder.java:190) +Method calls method in (KafkaSerializerWrapper.java:71) +Method calls method in (KafkaSerializerWrapper.java:88) +Method is annotated with in (KafkaSink.java:0) +Method calls method in (KafkaSinkBuilder.java:194) +Method calls method in (KafkaSinkBuilder.java:202) +Method calls method in (KafkaSinkBuilder.java:198) +Method calls method in (KafkaSinkBuilder.java:111) +Method calls method in (KafkaSinkBuilder.java:97) +Method calls method in (KafkaSinkBuilder.java:123) +Method calls method in (KafkaSinkBuilder.java:133) +Method calls method in (KafkaSinkBuilder.java:152) +Method calls method in (KafkaSinkBuilder.java:151) +Method calls method in (KafkaSinkBuilder.java:175) +Method calls method in (KafkaSinkBuilder.java:176) +Method calls method in (KafkaWriter.java:244) +Method calls method in (KafkaWriter.java:245) +Method calls method in (KafkaWriter.java:246) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (KafkaWriter.java:0) +Method calls method in (KafkaWriter.java:311) +Method is annotated with in (DynamicKafkaSource.java:0) +Method calls method in (DynamicKafkaSourceBuilder.java:290) +Method calls method in (DynamicKafkaSourceBuilder.java:292) +Method calls method in (DynamicKafkaSourceBuilder.java:294) +Method calls method in (DynamicKafkaSourceBuilder.java:298) +Method calls method in (DynamicKafkaSourceBuilder.java:99) +Method calls method in (DynamicKafkaSourceBuilder.java:72) +Method calls method in (DynamicKafkaSourceBuilder.java:85) +Method calls method in (DynamicKafkaSourceOptions.java:69) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method calls method in (KafkaSourceBuilder.java:514) +Method calls method in (KafkaSourceBuilder.java:519) +Method calls method in (KafkaSourceBuilder.java:523) +Method calls method in (KafkaSourceBuilder.java:525) +Method calls method in (KafkaSourceBuilder.java:200) +Method calls method in (YamlFileMetadataService.java:279) +Method calls method in (YamlFileMetadataService.java:277) +Method checks instanceof in (YamlFileMetadataService.java:276) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:283) +Method calls method in (YamlFileMetadataService.java:283) +Method has parameter of type in (YamlFileMetadataService.java:0) +Method calls constructor (java.lang.Class)> in (YamlFileMetadataService.java:260) +Method calls constructor (org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.constructor.BaseConstructor, org.apache.flink.shaded.jackson2.org.yaml.snakeyaml.representer.Representer)> in (YamlFileMetadataService.java:263) +Method calls constructor ()> in (YamlFileMetadataService.java:258) +Method calls method in (YamlFileMetadataService.java:259) +Method calls method in (YamlFileMetadataService.java:261) +Method calls method in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:262) +Method gets field in (YamlFileMetadataService.java:259) +Method has return type in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:220) +Method is annotated with in (YamlFileMetadataService.java:0) +Method calls method in (YamlFileMetadataService.java:165) +Method calls method in (DynamicKafkaSourceEnumStateSerializer.java:140) +Method calls method in (DynamicKafkaSourceEnumerator.java:506) +Method is annotated with in (KafkaSourceEnumStateSerializer.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method calls method in (StoppableKafkaEnumContextProxy.java:225) +Method calls method in (StoppableKafkaEnumContextProxy.java:247) +Method calls method in (ReaderHandledOffsetsInitializer.java:75) +Method calls method in (SpecifiedOffsetsInitializer.java:105) +Method calls method in (KafkaClusterMetricGroupManager.java:73) +Method calls method in (KafkaClusterMetricGroupManager.java:62) +Method checks instanceof in (KafkaClusterMetricGroupManager.java:42) +Method calls constructor (int)> in (DynamicKafkaSourceReader.java:474) +Method calls method in (DynamicKafkaSourceReader.java:473) +Method calls method in (DynamicKafkaSourceReader.java:488) +Method calls method in (DynamicKafkaSourceReader.java:417) +Method has return type in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:226) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:382) +Method calls method in (DynamicKafkaSourceReader.java:380) +Method calls method in (DynamicKafkaSourceReader.java:499) +Method calls method in (KafkaPartitionSplitReader.java:540) +Method calls method in (KafkaPartitionSplitReader.java:359) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method calls method in (KafkaValueOnlyDeserializerWrapper.java:65) +Method calls constructor (java.lang.Object)> in (FlinkKafkaConsumerBase.java:306) +Method calls method in (FlinkKafkaConsumerBase.java:304) +Method calls method in (FlinkKafkaConsumerBase.java:301) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks)> in (FlinkKafkaConsumerBase.java:398) +Method calls method in (FlinkKafkaConsumerBase.java:397) +Method calls method in (FlinkKafkaConsumerBase.java:390) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks)> in (FlinkKafkaConsumerBase.java:352) +Method calls method in (FlinkKafkaConsumerBase.java:351) +Method calls method in (FlinkKafkaConsumerBase.java:344) +Method has generic parameter type > with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:967) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method has parameter of type in (FlinkKafkaConsumerBase.java:0) +Method calls constructor (java.lang.Class, [Lorg.apache.flink.api.common.typeutils.TypeSerializer;)> in (FlinkKafkaConsumerBase.java:1225) +Method calls constructor (java.lang.Class, org.apache.flink.api.common.ExecutionConfig)> in (FlinkKafkaConsumerBase.java:1217) +Method gets field in (FlinkKafkaConsumerBase.java:1217) +Method has return type in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method is annotated with in (FlinkKafkaConsumerBase.java:0) +Method calls method in (FlinkKafkaConsumerBase.java:752) +Method calls method in (FlinkKafkaConsumerBase.java:566) +Method calls method in (FlinkKafkaConsumerBase.java:535) +Method calls method in (FlinkKafkaConsumerBase.java:472) +Method calls method in (FlinkKafkaConsumerBase.java:477) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:404) +Method is annotated with in (FlinkKafkaProducerBase.java:0) +Method calls method in (FlinkKafkaProducerBase.java:223) +Method calls method in (FlinkKafkaProducerBase.java:260) +Method calls method in (AbstractFetcher.java:604) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:608) +Method calls method in (AbstractFetcher.java:593) +Method calls method in (AbstractFetcher.java:593) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method calls method in (AbstractFetcher.java:430) +Method calls method in (AbstractFetcher.java:428) +Method calls method in (AbstractFetcher.java:426) +Method calls method in (AbstractFetcher.java:419) +Method has generic parameter type >> with type argument depending on in (AbstractFetcher.java:0) +Method has parameter of type in (AbstractFetcher.java:0) +Method is annotated with in (FlinkKafkaInternalProducer.java:0) +Method calls method in (FlinkKafkaInternalProducer.java:203) +Method calls method in (Handover.java:82) +Method calls method in (Handover.java:109) +Method calls method in (Handover.java:149) +Method is annotated with in (KafkaConsumerThread.java:0) +Method is annotated with in (KafkaConsumerThread.java:0) +Method calls method in (KafkaFetcher.java:223) +Method calls method in (KafkaFetcher.java:150) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:240) +Method calls method in (KafkaShuffleFetcher.java:244) +Method calls method in (KafkaShuffleFetcher.java:245) +Method calls method in (KafkaShuffleFetcher.java:254) +Method calls method in (KafkaShuffleFetcher.java:251) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:238) +Method gets field in (KafkaShuffleFetcher.java:244) +Method gets field in (KafkaShuffleFetcher.java:245) +Method gets field in (KafkaShuffleFetcher.java:253) +Method gets field in (KafkaShuffleFetcher.java:250) +Method gets field in (KafkaShuffleFetcher.java:254) +Method is annotated with in (KafkaShuffleFetcher.java:0) +Method calls method in (KafkaShuffleFetcher.java:279) +Method calls method in (FlinkFixedPartitioner.java:67) +Method calls method in (FlinkFixedPartitioner.java:69) +Method calls method in (FlinkFixedPartitioner.java:77) +Method calls constructor (org.apache.flink.api.dag.Transformation, java.lang.String, org.apache.flink.streaming.api.operators.StreamSink, int, boolean)> in (FlinkKafkaShuffle.java:380) +Method calls method in (FlinkKafkaShuffle.java:371) +Method calls method in (FlinkKafkaShuffle.java:383) +Method calls constructor ([I, org.apache.flink.api.common.typeinfo.TypeInformation)> in (FlinkKafkaShuffle.java:394) +Method calls method in (FlinkKafkaShuffle.java:392) +Method calls method in (FlinkKafkaShuffle.java:396) +Method calls method in (FlinkKafkaShuffle.java:151) +Method calls method in (FlinkKafkaShuffle.java:352) +Method calls method in (FlinkKafkaShuffle.java:340) +Method calls method in (FlinkKafkaShuffle.java:344) +Method calls method in (FlinkKafkaShuffle.java:256) +Method calls method in (FlinkKafkaShuffle.java:260) +Method calls method in (FlinkKafkaShuffleConsumer.java:81) +Method calls method in (FlinkKafkaShuffleConsumer.java:87) +Method calls method in (FlinkKafkaShuffleConsumer.java:80) +Method calls method in (FlinkKafkaShuffleConsumer.java:83) +Method calls method in (FlinkKafkaShuffleConsumer.java:82) +Method has generic parameter type >> with type argument depending on in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method has parameter of type in (FlinkKafkaShuffleConsumer.java:0) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:186) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:204) +Method calls method in (FlinkKafkaShuffleProducer.java:190) +Method calls method in (FlinkKafkaShuffleProducer.java:193) +Method calls method in (FlinkKafkaShuffleProducer.java:195) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:212) +Method calls method in (FlinkKafkaShuffleProducer.java:225) +Method calls method in (FlinkKafkaShuffleProducer.java:224) +Method calls method in (FlinkKafkaShuffleProducer.java:216) +Method calls method in (FlinkKafkaShuffleProducer.java:217) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:160) +Method calls method in (FlinkKafkaShuffleProducer.java:99) +Method calls method in (StreamKafkaShuffleSink.java:40) +Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (KafkaConnectorOptionsUtil.java:543) +Method calls method in (KafkaConnectorOptionsUtil.java:518) +Method calls method in (KafkaConnectorOptionsUtil.java:587) +Method calls method in (KafkaConnectorOptionsUtil.java:585) +Method calls method in (KafkaConnectorOptionsUtil.java:498) +Method calls method in (KafkaDynamicSink.java:386) +Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) +Method calls method in (KafkaDynamicSource.java:566) +Method calls method in (ReducingUpsertWriter.java:177) diff --git a/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 new file mode 100644 index 000000000..ebea539d5 --- /dev/null +++ b/flink-connector-kafka/archunit-violations/c0d94764-76a0-4c50-b617-70b1754c4612 @@ -0,0 +1,112 @@ +Class implements interface in (KafkaSink.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class is annotated with in (KafkaSourceEnumerator.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class extends class in (FlinkKafkaProducer.java:0) +Class is annotated with in (FlinkKafkaProducer.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Class is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber, org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService, org.apache.flink.api.connector.source.SplitEnumeratorContext, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer, java.util.Properties, org.apache.flink.api.connector.source.Boundedness, org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState, org.apache.flink.connector.kafka.dynamic.source.enumerator.StoppableKafkaEnumContextProxy$StoppableKafkaEnumContextProxyFactory)> is annotated with in (DynamicKafkaSourceEnumerator.java:0) +Constructor (org.apache.flink.api.connector.source.SourceReaderContext, org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema, java.util.Properties)> calls constructor (int)> in (DynamicKafkaSourceReader.java:114) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1710) +Constructor (java.util.Set)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, long, short, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (java.lang.String, org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor (org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer)> is annotated with in (FlinkKafkaProducer.java:0) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1848) +Constructor ()> calls constructor ()> in (FlinkKafkaProducer.java:1618) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (java.lang.Object, java.util.List, org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer, org.apache.flink.streaming.runtime.tasks.ProcessingTimeService, long)> has parameter of type in (AbstractFetcher.java:0) +Constructor (org.apache.flink.api.common.typeutils.TypeSerializer)> is annotated with in (KafkaShuffleFetcher.java:0) +Constructor (java.util.List, java.util.regex.Pattern, org.apache.flink.connector.kafka.sink.KafkaPartitioner, org.apache.flink.api.common.serialization.SerializationSchema, org.apache.flink.api.common.serialization.SerializationSchema, [Lorg.apache.flink.table.data.RowData$FieldGetter;, [Lorg.apache.flink.table.data.RowData$FieldGetter;, boolean, [I, boolean)> has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has generic type > with type argument depending on in (KafkaClusterMetricGroupManager.java:0) +Field has type in (DynamicKafkaSourceReader.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (AbstractFetcher.java:0) +Field has type in (KafkaShuffleFetcher.java:0) +Field has type in (FlinkKafkaShuffleProducer.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Field has type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method is annotated with in (DynamicKafkaSource.java:0) +Method calls method in (KafkaClusterMetricGroupManager.java:73) +Method calls method in (KafkaClusterMetricGroupManager.java:62) +Method checks instanceof in (KafkaClusterMetricGroupManager.java:42) +Method calls constructor (int)> in (DynamicKafkaSourceReader.java:475) +Method calls method in (DynamicKafkaSourceReader.java:474) +Method calls method in (DynamicKafkaSourceReader.java:485) +Method has return type in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method is annotated with in (DynamicKafkaSourceReader.java:0) +Method calls method in (DynamicKafkaSourceReader.java:383) +Method calls method in (DynamicKafkaSourceReader.java:381) +Method calls method in (DynamicKafkaSourceReader.java:496) +Method is annotated with in (ExactlyOnceKafkaWriter.java:0) +Method is annotated with in (ExactlyOnceKafkaWriter.java:0) +Method is annotated with in (KafkaCommitter.java:0) +Method calls method in (KafkaSink.java:168) +Method calls method in (KafkaSink.java:171) +Method calls method in (KafkaSink.java:167) +Method calls method in (KafkaSink.java:170) +Method checks instanceof in (KafkaSink.java:167) +Method has generic parameter type >> with type argument depending on in (KafkaSink.java:0) +Method is annotated with in (KafkaSink.java:0) +Method calls method in (KafkaSinkBuilder.java:152) +Method is annotated with in (KafkaWriter.java:0) +Method is annotated with in (ProducerPoolImpl.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSource.java:0) +Method is annotated with in (KafkaSourceEnumStateSerializer.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaSourceEnumerator.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaPartitionSplitReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method is annotated with in (KafkaSourceReader.java:0) +Method calls method in (AbstractFetcher.java:613) +Method calls method in (AbstractFetcher.java:617) +Method calls method in (AbstractFetcher.java:617) +Method calls method in (AbstractFetcher.java:602) +Method calls method in (AbstractFetcher.java:602) +Method calls constructor ([B)> in (KafkaShuffleFetcher.java:241) +Method calls method in (KafkaShuffleFetcher.java:245) +Method calls method in (KafkaShuffleFetcher.java:246) +Method calls method in (KafkaShuffleFetcher.java:255) +Method calls method in (KafkaShuffleFetcher.java:252) +Method calls method in (KafkaShuffleFetcher.java:256) +Method calls method in (KafkaShuffleFetcher.java:239) +Method gets field in (KafkaShuffleFetcher.java:245) +Method gets field in (KafkaShuffleFetcher.java:246) +Method gets field in (KafkaShuffleFetcher.java:254) +Method gets field in (KafkaShuffleFetcher.java:251) +Method gets field in (KafkaShuffleFetcher.java:255) +Method is annotated with in (KafkaShuffleFetcher.java:0) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:187) +Method calls method in (FlinkKafkaShuffleProducer.java:206) +Method calls method in (FlinkKafkaShuffleProducer.java:205) +Method calls method in (FlinkKafkaShuffleProducer.java:191) +Method calls method in (FlinkKafkaShuffleProducer.java:194) +Method calls method in (FlinkKafkaShuffleProducer.java:196) +Method calls method in (FlinkKafkaShuffleProducer.java:197) +Method calls constructor (int)> in (FlinkKafkaShuffleProducer.java:213) +Method calls method in (FlinkKafkaShuffleProducer.java:226) +Method calls method in (FlinkKafkaShuffleProducer.java:225) +Method calls method in (FlinkKafkaShuffleProducer.java:217) +Method calls method in (FlinkKafkaShuffleProducer.java:218) +Method calls method in (FlinkKafkaShuffleProducer.java:219) +Method calls method in (FlinkKafkaShuffleProducer.java:220) +Method has parameter of type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (DynamicKafkaRecordSerializationSchema.java:0) +Method calls method in (KafkaConnectorOptionsUtil.java:525) +Method calls method in (KafkaConnectorOptionsUtil.java:569) +Method calls method in (KafkaDynamicSink.java:401) +Method has return type <[Lorg.apache.flink.table.data.RowData$FieldGetter;> in (KafkaDynamicSink.java:0) +Method calls method in (KafkaDynamicSource.java:566) diff --git a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d index b591e33c6..09a12f009 100644 --- a/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d +++ b/flink-connector-kafka/archunit-violations/d853eb69-8c04-4246-9a5e-4f5911286b1d @@ -1,5 +1,7 @@ org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.open(org.apache.flink.api.common.serialization.SerializationSchema$InitializationContext, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema.serialize(java.lang.Object, org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext, java.lang.Long): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema$KafkaSinkContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.connector.kafka.sink.KafkaCommittable does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.addPostCommitTopology(org.apache.flink.streaming.api.datastream.DataStream): Argument leaf type org.apache.flink.streaming.api.connector.sink2.CommittableMessage does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated +org.apache.flink.connector.kafka.sink.KafkaSink.createCommitter(org.apache.flink.api.connector.sink2.CommitterInitContext): Returned leaf type org.apache.flink.connector.kafka.sink.KafkaCommittable does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer.getPartitionOffsets(java.util.Collection, org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever): Argument leaf type org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer$PartitionOffsetsRetriever does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.invoke(org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState, java.lang.Object, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context): Argument leaf type org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$KafkaTransactionState does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated -org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition.dropLeaderData(java.util.List): Argument leaf type org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated \ No newline at end of file diff --git a/flink-connector-kafka/archunit-violations/stored.rules b/flink-connector-kafka/archunit-violations/stored.rules index 372199016..122522db7 100644 --- a/flink-connector-kafka/archunit-violations/stored.rules +++ b/flink-connector-kafka/archunit-violations/stored.rules @@ -1,10 +1,16 @@ # -#Fri Feb 10 21:57:58 CET 2023 +#Sat Aug 10 18:37:22 PDT 2024 Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=690729ae-756f-4ccc-831b-2667db3fdae5 Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=3030609a-c417-456d-af43-1799a4445197 +Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=661bd4bd-792c-4ba4-af39-87b7d16dcf75 ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=86dfd459-67a9-4b26-9b5c-0b0bbf22681a Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=27a0a5e4-29c2-4069-b381-952746c90862 Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=3b06cb2f-1036-4d54-a358-c37f163611ee Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=cdd3929e-b7bc-4541-a0ae-46ed4fbb2001 +Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=3b06cb2f-1036-4d54-a358-c37f163611ee +Connector\ production\ code\ must\ not\ depend\ on\ non-public\ API\ outside\ of\ connector\ packages=3030609a-c417-456d-af43-1799a4445197 +Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=27a0a5e4-29c2-4069-b381-952746c90862 +Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=c0d94764-76a0-4c50-b617-70b1754c4612 +Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=690729ae-756f-4ccc-831b-2667db3fdae5 Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=d853eb69-8c04-4246-9a5e-4f5911286b1d Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=661bd4bd-792c-4ba4-af39-87b7d16dcf75 diff --git a/flink-connector-kafka/pom.xml b/flink-connector-kafka/pom.xml index 0355775c0..ae4cc71b8 100644 --- a/flink-connector-kafka/pom.xml +++ b/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 3.4-SNAPSHOT flink-connector-kafka @@ -35,8 +35,12 @@ under the License. jar - 2.12 - 2.0.9 + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + @@ -56,6 +60,7 @@ under the License. org.apache.flink flink-connector-base ${flink.version} + provided @@ -77,6 +82,11 @@ under the License. ${kafka.version} + + com.google.guava + guava + + @@ -124,37 +134,22 @@ under the License. test-jar - - - org.apache.kafka - kafka_${scala.binary.version} - ${kafka.version} - - - org.slf4j - slf4j-api - - + org.testcontainers + kafka test - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - test - - org.testcontainers - kafka + junit-jupiter test - org.testcontainers - junit-jupiter + org.yaml + snakeyaml + 2.2 test @@ -180,6 +175,14 @@ under the License. test + + org.apache.flink + flink-connector-base + ${flink.version} + test + test-jar + + org.apache.flink flink-metrics-jmx @@ -209,6 +212,7 @@ under the License. ${flink.version} test + org.apache.flink flink-table-planner_${scala.binary.version} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java new file mode 100644 index 000000000..1ff18930e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/ClusterMetadata.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +/** + * {@link ClusterMetadata} provides readers information about a cluster on what topics to read and + * how to connect to a cluster. + */ +@Experimental +public class ClusterMetadata implements Serializable { + private final Set topics; + private final Properties properties; + + /** + * Constructs the {@link ClusterMetadata} with the required properties. + * + * @param topics the topics belonging to a cluster. + * @param properties the properties to access a cluster. + */ + public ClusterMetadata(Set topics, Properties properties) { + this.topics = topics; + this.properties = properties; + } + + /** + * Get the topics. + * + * @return the topics. + */ + public Set getTopics() { + return topics; + } + + /** + * Get the properties. + * + * @return the properties. + */ + public Properties getProperties() { + return properties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("topics", topics) + .add("properties", properties) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ClusterMetadata that = (ClusterMetadata) o; + return Objects.equals(topics, that.topics) && Objects.equals(properties, that.properties); + } + + @Override + public int hashCode() { + return Objects.hash(topics, properties); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java new file mode 100644 index 000000000..179277dad --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaMetadataService.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +/** Metadata service that returns Kafka details. */ +@Experimental +public interface KafkaMetadataService extends AutoCloseable, Serializable { + /** + * Get current metadata for all streams. + * + * @return set of all streams + */ + Set getAllStreams(); + + /** + * Get current metadata for queried streams. + * + * @param streamIds stream full names + * @return map of stream name to metadata + */ + Map describeStreams(Collection streamIds); + + /** + * Check if the cluster is active. + * + * @param kafkaClusterId Kafka cluster id + * @return boolean whether the cluster is active + */ + boolean isClusterActive(String kafkaClusterId); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java new file mode 100644 index 000000000..bea9872c7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/KafkaStream.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; + +import com.google.common.base.MoreObjects; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +/** + * Kafka stream represents multiple topics over multiple Kafka clusters and this class encapsulates + * all the necessary information to initiate Kafka consumers to read a stream. + */ +@Experimental +public class KafkaStream implements Serializable { + private final String streamId; + private final Map clusterMetadataMap; + + /** + * Construct a {@link KafkaStream} by passing Kafka information in order to connect to the + * stream. + * + * @param streamId the stream id. + * @param clusterMetadataMap the map of clusters to {@link ClusterMetadata} to connect to the + * stream. + */ + public KafkaStream(String streamId, Map clusterMetadataMap) { + this.streamId = streamId; + this.clusterMetadataMap = clusterMetadataMap; + } + + /** + * Get the stream id. + * + * @return the stream id. + */ + public String getStreamId() { + return streamId; + } + + /** + * Get the metadata to connect to the various cluster(s). + * + * @return the cluster metadata map. + */ + public Map getClusterMetadataMap() { + return clusterMetadataMap; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataMap", clusterMetadataMap) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaStream that = (KafkaStream) o; + return Objects.equals(streamId, that.streamId) + && Objects.equals(clusterMetadataMap, that.clusterMetadataMap); + } + + @Override + public int hashCode() { + return Objects.hash(streamId, clusterMetadataMap); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java new file mode 100644 index 000000000..6cef3ab31 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/metadata/SingleClusterTopicMetadataService.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.metadata; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +/** + * A {@link KafkaMetadataService} that delegates metadata fetching to a single {@link AdminClient}, + * which is scoped to a single cluster. The stream ids are equivalent to topics. + */ +@Experimental +public class SingleClusterTopicMetadataService implements KafkaMetadataService { + + private final String kafkaClusterId; + private final Properties properties; + private transient AdminClient adminClient; + + /** + * Create a {@link SingleClusterTopicMetadataService}. + * + * @param kafkaClusterId the id of the Kafka cluster. + * @param properties the properties of the Kafka cluster. + */ + public SingleClusterTopicMetadataService(String kafkaClusterId, Properties properties) { + this.kafkaClusterId = kafkaClusterId; + this.properties = properties; + } + + /** {@inheritDoc} */ + @Override + public Set getAllStreams() { + try { + return getAdminClient().listTopics().names().get().stream() + .map(this::createKafkaStream) + .collect(Collectors.toSet()); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + try { + return getAdminClient().describeTopics(new ArrayList<>(streamIds)).all().get().keySet() + .stream() + .collect(Collectors.toMap(topic -> topic, this::createKafkaStream)); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Fetching all streams failed", e); + } + } + + private KafkaStream createKafkaStream(String topic) { + ClusterMetadata clusterMetadata = + new ClusterMetadata(Collections.singleton(topic), properties); + + return new KafkaStream(topic, Collections.singletonMap(kafkaClusterId, clusterMetadata)); + } + + private AdminClient getAdminClient() { + if (adminClient == null) { + Properties adminClientProps = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, adminClientProps); + String clientIdPrefix = + adminClientProps.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + adminClientProps.setProperty( + CommonClientConfigs.CLIENT_ID_CONFIG, + clientIdPrefix + "-single-cluster-topic-metadata-service"); + adminClient = AdminClient.create(adminClientProps); + } + + return adminClient; + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return this.kafkaClusterId.equals(kafkaClusterId); + } + + /** {@inheritDoc} */ + @Override + public void close() { + if (adminClient != null) { + adminClient.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java new file mode 100644 index 000000000..9a93d7b88 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSource.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumState; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumerator; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.Properties; + +/** + * Factory class for the DynamicKafkaSource components. FLIP-246: DynamicKafkaSource + * + *

This source's key difference from {@link KafkaSource} is that it enables users to read + * dynamically, which does not require job restart, from streams (topics that belong to one or more + * clusters). If using {@link KafkaSource}, users need to restart the job by deleting the job and + * reconfiguring the topics and clusters. + * + *

This example shows how to configure a {@link DynamicKafkaSource} that emits Integer records: + * + *

{@code
+ * DynamicKafkaSource dynamicKafkaSource =
+ *                     DynamicKafkaSource.builder()
+ *                             .setStreamIds(Collections.singleton("MY_STREAM_ID"))
+ *                             // custom metadata service that resolves `MY_STREAM_ID` to the associated clusters and topics
+ *                             .setKafkaMetadataService(kafkaMetadataService)
+ *                             .setDeserializer(
+ *                                     KafkaRecordDeserializationSchema.valueOnly(
+ *                                             IntegerDeserializer.class))
+ *                             .setStartingOffsets(OffsetsInitializer.earliest())
+ *                             // common properties for all Kafka clusters
+ *                             .setProperties(properties)
+ *                             .build();
+ * }
+ * + *

See more configuration options in {@link DynamicKafkaSourceBuilder} and {@link + * DynamicKafkaSourceOptions}. + * + * @param Record type + */ +@Experimental +public class DynamicKafkaSource + implements Source, + ResultTypeQueryable { + + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final KafkaMetadataService kafkaMetadataService; + private final KafkaRecordDeserializationSchema deserializationSchema; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetsInitializer; + private final Properties properties; + private final Boundedness boundedness; + + DynamicKafkaSource( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + KafkaRecordDeserializationSchema deserializationSchema, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetsInitializer, + Properties properties, + Boundedness boundedness) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaMetadataService = kafkaMetadataService; + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + this.boundedness = boundedness; + } + + /** + * Get a builder for this source. + * + * @return a {@link DynamicKafkaSourceBuilder}. + */ + public static DynamicKafkaSourceBuilder builder() { + return new DynamicKafkaSourceBuilder<>(); + } + + /** + * Get the {@link Boundedness}. + * + * @return the {@link Boundedness}. + */ + @Override + public Boundedness getBoundedness() { + return boundedness; + } + + /** + * Create the {@link DynamicKafkaSourceReader}. + * + * @param readerContext The {@link SourceReaderContext context} for the source reader. + * @return the {@link DynamicKafkaSourceReader}. + */ + @Internal + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new DynamicKafkaSourceReader<>(readerContext, deserializationSchema, properties); + } + + /** + * Create the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + new DynamicKafkaSourceEnumState()); + } + + /** + * Restore the {@link DynamicKafkaSourceEnumerator}. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the restored split + * enumerator. + * @param checkpoint The checkpoint to restore the SplitEnumerator from. + * @return the {@link DynamicKafkaSourceEnumerator}. + */ + @Internal + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + DynamicKafkaSourceEnumState checkpoint) { + return new DynamicKafkaSourceEnumerator( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + properties, + boundedness, + checkpoint); + } + + /** + * Get the {@link DynamicKafkaSourceSplitSerializer}. + * + * @return the {@link DynamicKafkaSourceSplitSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new DynamicKafkaSourceSplitSerializer(); + } + + /** + * Get the {@link DynamicKafkaSourceEnumStateSerializer}. + * + * @return the {@link DynamicKafkaSourceEnumStateSerializer}. + */ + @Internal + @Override + public SimpleVersionedSerializer + getEnumeratorCheckpointSerializer() { + return new DynamicKafkaSourceEnumStateSerializer(); + } + + /** + * Get the {@link TypeInformation} of the source. + * + * @return the {@link TypeInformation}. + */ + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + @VisibleForTesting + public KafkaStreamSubscriber getKafkaStreamSubscriber() { + return kafkaStreamSubscriber; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java new file mode 100644 index 000000000..eab37c4ee --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceBuilder.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.StreamPatternSubscriber; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; + +/** A builder class to make it easier for users to construct a {@link DynamicKafkaSource}. */ +@Experimental +public class DynamicKafkaSourceBuilder { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceBuilder.class); + private KafkaStreamSubscriber kafkaStreamSubscriber; + private KafkaMetadataService kafkaMetadataService; + private KafkaRecordDeserializationSchema deserializationSchema; + private OffsetsInitializer startingOffsetsInitializer; + private OffsetsInitializer stoppingOffsetsInitializer; + private Boundedness boundedness; + private final Properties props; + + DynamicKafkaSourceBuilder() { + this.kafkaStreamSubscriber = null; + this.kafkaMetadataService = null; + this.deserializationSchema = null; + this.startingOffsetsInitializer = OffsetsInitializer.earliest(); + this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.props = new Properties(); + } + + /** + * Set the stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamIds the stream ids. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamIds(Set streamIds) { + Preconditions.checkNotNull(streamIds); + ensureSubscriberIsNull("streamIds"); + this.kafkaStreamSubscriber = new KafkaStreamSetSubscriber(streamIds); + return this; + } + + /** + * Set the stream pattern to determine stream ids belonging to the {@link KafkaMetadataService}. + * + * @param streamPattern the stream pattern. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStreamPattern(Pattern streamPattern) { + Preconditions.checkNotNull(streamPattern); + ensureSubscriberIsNull("stream pattern"); + this.kafkaStreamSubscriber = new StreamPatternSubscriber(streamPattern); + return this; + } + + /** + * Set a custom Kafka stream subscriber. + * + * @param kafkaStreamSubscriber the {@link KafkaStreamSubscriber}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaStreamSubscriber( + KafkaStreamSubscriber kafkaStreamSubscriber) { + Preconditions.checkNotNull(kafkaStreamSubscriber); + ensureSubscriberIsNull("custom"); + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + return this; + } + + /** + * Set the source in bounded mode and specify what offsets to end at. This is used for all + * clusters. + * + * @param stoppingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setBounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.BOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * Set the {@link KafkaMetadataService}. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setKafkaMetadataService( + KafkaMetadataService kafkaMetadataService) { + this.kafkaMetadataService = kafkaMetadataService; + return this; + } + + /** + * Set the {@link KafkaRecordDeserializationSchema}. + * + * @param recordDeserializer the {@link KafkaRecordDeserializationSchema}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setDeserializer( + KafkaRecordDeserializationSchema recordDeserializer) { + this.deserializationSchema = recordDeserializer; + return this; + } + + /** + * Set the starting offsets of the stream. This will be applied to all clusters. + * + * @param startingOffsetsInitializer the {@link OffsetsInitializer}. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setStartingOffsets( + OffsetsInitializer startingOffsetsInitializer) { + this.startingOffsetsInitializer = startingOffsetsInitializer; + return this; + } + + /** + * Set the properties of the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param properties the properties. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperties(Properties properties) { + this.props.putAll(properties); + return this; + } + + /** + * Set a property for the consumers. This will be applied to all clusters and properties like + * {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} may be overriden by the {@link + * KafkaMetadataService}. + * + * @param key the property key. + * @param value the properties value. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setProperty(String key, String value) { + this.props.setProperty(key, value); + return this; + } + + /** + * Set the property for {@link CommonClientConfigs#GROUP_ID_CONFIG}. This will be applied to all + * clusters. + * + * @param groupId the group id. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setGroupId(String groupId) { + return setProperty(CommonClientConfigs.GROUP_ID_CONFIG, groupId); + } + + /** + * Set the client id prefix. This applies {@link KafkaSourceOptions#CLIENT_ID_PREFIX} to all + * clusters. + * + * @param prefix the client id prefix. + * @return the builder. + */ + public DynamicKafkaSourceBuilder setClientIdPrefix(String prefix) { + return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); + } + + /** + * Construct the source with the configuration that was set. + * + * @return the {@link DynamicKafkaSource}. + */ + public DynamicKafkaSource build() { + logger.info("Building the DynamicKafkaSource"); + sanityCheck(); + setRequiredConsumerProperties(); + return new DynamicKafkaSource<>( + kafkaStreamSubscriber, + kafkaMetadataService, + deserializationSchema, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + boundedness); + } + + // Below are utility methods, code and structure are mostly copied over from KafkaSourceBuilder + + private void setRequiredConsumerProperties() { + maybeOverride( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + maybeOverride( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + logger.warn( + "Offset commit on checkpoint is disabled because {} is not specified", + ConsumerConfig.GROUP_ID_CONFIG); + maybeOverride(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false", false); + } + maybeOverride(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false", false); + maybeOverride( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase(), + true); + + // If the source is bounded, do not run periodic partition discovery. + maybeOverride( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + + // If the source is bounded, do not run periodic metadata discovery + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + maybeOverride( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "0", + boundedness == Boundedness.BOUNDED); + + // If the client id prefix is not set, reuse the consumer group id as the client id prefix, + // or generate a random string if consumer group id is not specified. + maybeOverride( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) + ? props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) + : "DynamicKafkaSource-" + RandomStringUtils.randomAlphabetic(8), + false); + } + + private boolean maybeOverride(String key, String value, boolean override) { + boolean overridden = false; + String userValue = props.getProperty(key); + if (userValue != null) { + if (override) { + logger.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + props.setProperty(key, value); + overridden = true; + } + } else { + props.setProperty(key, value); + } + return overridden; + } + + private void sanityCheck() { + Preconditions.checkNotNull( + kafkaStreamSubscriber, "Kafka stream subscriber is required but not provided"); + Preconditions.checkNotNull( + kafkaMetadataService, "Kafka Metadata Service is required but not provided"); + Preconditions.checkNotNull( + deserializationSchema, "Deserialization schema is required but not provided."); + + // Check consumer group ID + Preconditions.checkState( + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(), + String.format( + "Property %s is required when offset commit is enabled", + ConsumerConfig.GROUP_ID_CONFIG)); + } + + private boolean offsetCommitEnabledManually() { + boolean autoCommit = + props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + && Boolean.parseBoolean( + props.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + boolean commitOnCheckpoint = + props.containsKey(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key()) + && Boolean.parseBoolean( + props.getProperty( + KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key())); + return autoCommit || commitOnCheckpoint; + } + + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (kafkaStreamSubscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, + kafkaStreamSubscriber.getClass().getSimpleName())); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java new file mode 100644 index 000000000..bdecaf390 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceOptions.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.Properties; +import java.util.function.Function; + +/** + * The connector options for {@link DynamicKafkaSource} that can be passed through the source + * properties e.g. {@link DynamicKafkaSourceBuilder#setProperties(Properties)}. + */ +@Internal +public class DynamicKafkaSourceOptions { + + private DynamicKafkaSourceOptions() {} + + public static final ConfigOption STREAM_METADATA_DISCOVERY_INTERVAL_MS = + ConfigOptions.key("stream-metadata-discovery-interval-ms") + .longType() + .defaultValue(-1L) + .withDescription( + "The interval in milliseconds for the source to discover " + + "the changes in stream metadata. A non-positive value disables the stream metadata discovery."); + + public static final ConfigOption STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD = + ConfigOptions.key("stream-metadata-discovery-failure-threshold") + .intType() + .defaultValue(1) + .withDescription( + "The number of consecutive failures before letting the exception from Kafka metadata service discovery " + + "trigger jobmanager failure and global failover. The default is one to at least catch startup " + + "failures."); + + @Internal + public static T getOption( + Properties props, ConfigOption configOption, Function parser) { + String value = props.getProperty(configOption.key()); + return (T) (value == null ? configOption.defaultValue() : parser.apply(value)); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java new file mode 100644 index 000000000..4d7ff7b2d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/GetMetadataUpdateEvent.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** Event to signal to enumerator that a reader needs to know the current metadata. */ +@Internal +public class GetMetadataUpdateEvent implements SourceEvent {} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java new file mode 100644 index 000000000..09b32d031 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/MetadataUpdateEvent.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; +import java.util.Set; + +/** + * Signals {@link DynamicKafkaSourceReader} to stop their underlying readers. The restart process is + * as follows: + * + *

1. Detect metadata change in enumerator 2. Stop sub enumerators and don't snapshot state 3. + * Send this event to all readers 4. Stop sub readers and snapshot state (offsets) 5. Start new sub + * enumerators with clean state and do total split reassignment to readers 6. Readers obtain splits, + * starting sub readers dynamically, and do reconciliation of starting offsets with the cached + * offsets + * + *

We don't snapshot enumerator state because we want to reassign previously assigned splits. + * After restart, readers need to reinitialize the sub readers by using the received splits. + */ +@Internal +public class MetadataUpdateEvent implements SourceEvent { + private final Set kafkaStreams; + + public MetadataUpdateEvent(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("kafkaStreams", kafkaStreams).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MetadataUpdateEvent that = (MetadataUpdateEvent) o; + return Objects.equals(kafkaStreams, that.kafkaStreams); + } + + @Override + public int hashCode() { + return Objects.hash(kafkaStreams); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java new file mode 100644 index 000000000..0dd0a7744 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumState.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * The enumerator state keeps track of the state of the sub enumerators assigned splits and + * metadata. + */ +@Internal +public class DynamicKafkaSourceEnumState { + private final Set kafkaStreams; + private final Map clusterEnumeratorStates; + + public DynamicKafkaSourceEnumState() { + this.kafkaStreams = new HashSet<>(); + this.clusterEnumeratorStates = new HashMap<>(); + } + + public DynamicKafkaSourceEnumState( + Set kafkaStreams, + Map clusterEnumeratorStates) { + this.kafkaStreams = kafkaStreams; + this.clusterEnumeratorStates = clusterEnumeratorStates; + } + + public Set getKafkaStreams() { + return kafkaStreams; + } + + public Map getClusterEnumeratorStates() { + return clusterEnumeratorStates; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java new file mode 100644 index 000000000..b34e536c4 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializer.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.Preconditions; + +import org.apache.kafka.clients.CommonClientConfigs; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** (De)serializer for {@link DynamicKafkaSourceEnumState}. */ +@Internal +public class DynamicKafkaSourceEnumStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaSourceEnumStateSerializer kafkaSourceEnumStateSerializer; + + public DynamicKafkaSourceEnumStateSerializer() { + this.kafkaSourceEnumStateSerializer = new KafkaSourceEnumStateSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceEnumState state) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + Set kafkaStreams = state.getKafkaStreams(); + serialize(kafkaStreams, out); + + Map clusterEnumeratorStates = + state.getClusterEnumeratorStates(); + out.writeInt(kafkaSourceEnumStateSerializer.getVersion()); + + // write sub enumerator states + out.writeInt(clusterEnumeratorStates.size()); + for (Map.Entry clusterEnumeratorState : + clusterEnumeratorStates.entrySet()) { + String kafkaClusterId = clusterEnumeratorState.getKey(); + out.writeUTF(kafkaClusterId); + byte[] bytes = + kafkaSourceEnumStateSerializer.serialize(clusterEnumeratorState.getValue()); + // we need to know the exact size of the byte array since + // KafkaSourceEnumStateSerializer + // will throw exception if there are leftover unread bytes in deserialization. + out.writeInt(bytes.length); + out.write(bytes); + } + + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceEnumState deserialize(int version, byte[] serialized) + throws IOException { + if (version == VERSION_1) { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + Set kafkaStreams = deserialize(in); + + Map clusterEnumeratorStates = new HashMap<>(); + int kafkaSourceEnumStateSerializerVersion = in.readInt(); + + int clusterEnumeratorStateMapSize = in.readInt(); + for (int i = 0; i < clusterEnumeratorStateMapSize; i++) { + String kafkaClusterId = in.readUTF(); + int byteArraySize = in.readInt(); + KafkaSourceEnumState kafkaSourceEnumState = + kafkaSourceEnumStateSerializer.deserialize( + kafkaSourceEnumStateSerializerVersion, + readNBytes(in, byteArraySize)); + clusterEnumeratorStates.put(kafkaClusterId, kafkaSourceEnumState); + } + + return new DynamicKafkaSourceEnumState(kafkaStreams, clusterEnumeratorStates); + } + } + + throw new IOException( + String.format( + "The bytes are serialized with version %d, " + + "while this deserializer only supports version up to %d", + version, getVersion())); + } + + private void serialize(Set kafkaStreams, DataOutputStream out) throws IOException { + out.writeInt(kafkaStreams.size()); + for (KafkaStream kafkaStream : kafkaStreams) { + out.writeUTF(kafkaStream.getStreamId()); + Map clusterMetadataMap = kafkaStream.getClusterMetadataMap(); + out.writeInt(clusterMetadataMap.size()); + for (Map.Entry entry : clusterMetadataMap.entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + out.writeUTF(kafkaClusterId); + out.writeInt(clusterMetadata.getTopics().size()); + for (String topic : clusterMetadata.getTopics()) { + out.writeUTF(topic); + } + + // only write bootstrap server for now, can extend later to serialize the complete + // properties + out.writeUTF( + Preconditions.checkNotNull( + clusterMetadata + .getProperties() + .getProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + "Bootstrap servers must be specified in properties"))); + } + } + } + + private Set deserialize(DataInputStream in) throws IOException { + + Set kafkaStreams = new HashSet<>(); + int numStreams = in.readInt(); + for (int i = 0; i < numStreams; i++) { + String streamId = in.readUTF(); + Map clusterMetadataMap = new HashMap<>(); + int clusterMetadataMapSize = in.readInt(); + for (int j = 0; j < clusterMetadataMapSize; j++) { + String kafkaClusterId = in.readUTF(); + int topicsSize = in.readInt(); + Set topics = new HashSet<>(); + for (int k = 0; k < topicsSize; k++) { + topics.add(in.readUTF()); + } + + String bootstrapServers = in.readUTF(); + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + clusterMetadataMap.put(kafkaClusterId, new ClusterMetadata(topics, properties)); + } + + kafkaStreams.add(new KafkaStream(streamId, clusterMetadataMap)); + } + + return kafkaStreams; + } + + private static byte[] readNBytes(DataInputStream in, int size) throws IOException { + byte[] bytes = new byte[size]; + in.readFully(bytes); + return bytes; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java new file mode 100644 index 000000000..b61cee403 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumerator.java @@ -0,0 +1,545 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.util.Preconditions; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.MapDifference; +import com.google.common.collect.Maps; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +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.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This enumerator manages multiple {@link KafkaSourceEnumerator}'s, which does not have any + * synchronization since it assumes single threaded execution. + */ +@Internal +public class DynamicKafkaSourceEnumerator + implements SplitEnumerator { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceEnumerator.class); + + // Each cluster will have its own sub enumerator + private final Map> + clusterEnumeratorMap; + + // The mapping that the split enumerator context needs to be able to forward certain requests. + private final Map clusterEnumContextMap; + private final KafkaStreamSubscriber kafkaStreamSubscriber; + private final SplitEnumeratorContext enumContext; + private final KafkaMetadataService kafkaMetadataService; + private final Properties properties; + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetInitializer; + private final Boundedness boundedness; + private final StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory; + + // options + private final long kafkaMetadataServiceDiscoveryIntervalMs; + private final int kafkaMetadataServiceDiscoveryFailureThreshold; + + // state + private int kafkaMetadataServiceDiscoveryFailureCount; + private Map> latestClusterTopicsMap; + private Set latestKafkaStreams; + private boolean firstDiscoveryComplete; + + public DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState) { + this( + kafkaStreamSubscriber, + kafkaMetadataService, + enumContext, + startingOffsetsInitializer, + stoppingOffsetInitializer, + properties, + boundedness, + dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + .getDefaultFactory()); + } + + @VisibleForTesting + DynamicKafkaSourceEnumerator( + KafkaStreamSubscriber kafkaStreamSubscriber, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + OffsetsInitializer startingOffsetsInitializer, + OffsetsInitializer stoppingOffsetInitializer, + Properties properties, + Boundedness boundedness, + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState, + StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory + stoppableKafkaEnumContextProxyFactory) { + this.kafkaStreamSubscriber = kafkaStreamSubscriber; + this.boundedness = boundedness; + + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.properties = properties; + this.enumContext = enumContext; + + // options + this.kafkaMetadataServiceDiscoveryIntervalMs = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS, + Long::parseLong); + this.kafkaMetadataServiceDiscoveryFailureThreshold = + DynamicKafkaSourceOptions.getOption( + properties, + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD, + Integer::parseInt); + this.kafkaMetadataServiceDiscoveryFailureCount = 0; + this.firstDiscoveryComplete = false; + + this.kafkaMetadataService = kafkaMetadataService; + this.stoppableKafkaEnumContextProxyFactory = stoppableKafkaEnumContextProxyFactory; + + // handle checkpoint state and rebuild contexts + this.clusterEnumeratorMap = new HashMap<>(); + this.clusterEnumContextMap = new HashMap<>(); + this.latestKafkaStreams = dynamicKafkaSourceEnumState.getKafkaStreams(); + + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : latestKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterProperties.put(entry.getKey(), entry.getValue().getProperties()); + } + } + + this.latestClusterTopicsMap = new HashMap<>(); + for (Entry clusterEnumState : + dynamicKafkaSourceEnumState.getClusterEnumeratorStates().entrySet()) { + this.latestClusterTopicsMap.put( + clusterEnumState.getKey(), + clusterEnumState.getValue().assignedPartitions().stream() + .map(TopicPartition::topic) + .collect(Collectors.toSet())); + + createEnumeratorWithAssignedTopicPartitions( + clusterEnumState.getKey(), + this.latestClusterTopicsMap.get(clusterEnumState.getKey()), + clusterEnumState.getValue(), + clusterProperties.get(clusterEnumState.getKey())); + } + } + + /** + * Discover Kafka clusters and initialize sub enumerators. Bypass kafka metadata service + * discovery if there exists prior state. Exceptions with initializing Kafka source are treated + * the same as Kafka state and metadata inconsistency. + */ + @Override + public void start() { + // if there is checkpoint state, start all enumerators first. + if (!clusterEnumeratorMap.isEmpty()) { + startAllEnumerators(); + } + + if (kafkaMetadataServiceDiscoveryIntervalMs <= 0) { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch); + } else { + enumContext.callAsync( + () -> kafkaStreamSubscriber.getSubscribedStreams(kafkaMetadataService), + this::onHandleSubscribedStreamsFetch, + 0, + kafkaMetadataServiceDiscoveryIntervalMs); + } + } + + private void handleNoMoreSplits() { + if (Boundedness.BOUNDED.equals(boundedness)) { + boolean allEnumeratorsHaveSignalledNoMoreSplits = true; + for (StoppableKafkaEnumContextProxy context : clusterEnumContextMap.values()) { + allEnumeratorsHaveSignalledNoMoreSplits = + allEnumeratorsHaveSignalledNoMoreSplits && context.isNoMoreSplits(); + } + + if (firstDiscoveryComplete && allEnumeratorsHaveSignalledNoMoreSplits) { + logger.info( + "Signal no more splits to all readers: {}", + enumContext.registeredReaders().keySet()); + enumContext.registeredReaders().keySet().forEach(enumContext::signalNoMoreSplits); + } else { + logger.info("Not ready to notify no more splits to readers."); + } + } + } + + // --------------- private methods for metadata discovery --------------- + + private void onHandleSubscribedStreamsFetch(Set fetchedKafkaStreams, Throwable t) { + firstDiscoveryComplete = true; + Set handledFetchKafkaStreams = + handleFetchSubscribedStreamsError(fetchedKafkaStreams, t); + + Map> newClustersTopicsMap = new HashMap<>(); + Map clusterProperties = new HashMap<>(); + for (KafkaStream kafkaStream : handledFetchKafkaStreams) { + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + String kafkaClusterId = entry.getKey(); + ClusterMetadata clusterMetadata = entry.getValue(); + + newClustersTopicsMap + .computeIfAbsent(kafkaClusterId, (unused) -> new HashSet<>()) + .addAll(clusterMetadata.getTopics()); + clusterProperties.put(kafkaClusterId, clusterMetadata.getProperties()); + } + } + + // don't do anything if no change + if (latestClusterTopicsMap.equals(newClustersTopicsMap)) { + return; + } + + if (logger.isInfoEnabled()) { + MapDifference> metadataDifference = + Maps.difference(latestClusterTopicsMap, newClustersTopicsMap); + logger.info( + "Common cluster topics after metadata refresh: {}", + metadataDifference.entriesInCommon()); + logger.info( + "Removed cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnLeft()); + logger.info( + "Additional cluster topics after metadata refresh: {}", + metadataDifference.entriesOnlyOnRight()); + } + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState; + try { + dynamicKafkaSourceEnumState = snapshotState(-1); + } catch (Exception e) { + throw new RuntimeException("unable to snapshot state in metadata change", e); + } + + logger.info("Closing enumerators due to metadata change"); + + closeAllEnumeratorsAndContexts(); + latestClusterTopicsMap = newClustersTopicsMap; + latestKafkaStreams = handledFetchKafkaStreams; + sendMetadataUpdateEventToAvailableReaders(); + + // create enumerators + for (Entry> activeClusterTopics : latestClusterTopicsMap.entrySet()) { + KafkaSourceEnumState kafkaSourceEnumState = + dynamicKafkaSourceEnumState + .getClusterEnumeratorStates() + .get(activeClusterTopics.getKey()); + + final KafkaSourceEnumState newKafkaSourceEnumState; + if (kafkaSourceEnumState != null) { + final Set activeTopics = activeClusterTopics.getValue(); + + // filter out removed topics + Set partitions = + kafkaSourceEnumState.partitions().stream() + .filter(tp -> activeTopics.contains(tp.topicPartition().topic())) + .collect(Collectors.toSet()); + + newKafkaSourceEnumState = + new KafkaSourceEnumState( + partitions, kafkaSourceEnumState.initialDiscoveryFinished()); + } else { + newKafkaSourceEnumState = new KafkaSourceEnumState(Collections.emptySet(), false); + } + + // restarts enumerator from state using only the active topic partitions, to avoid + // sending duplicate splits from enumerator + createEnumeratorWithAssignedTopicPartitions( + activeClusterTopics.getKey(), + activeClusterTopics.getValue(), + newKafkaSourceEnumState, + clusterProperties.get(activeClusterTopics.getKey())); + } + + startAllEnumerators(); + } + + private Set handleFetchSubscribedStreamsError( + Set fetchedKafkaStreams, @Nullable Throwable t) { + if (t != null) { + if (!latestKafkaStreams.isEmpty() + && ++kafkaMetadataServiceDiscoveryFailureCount + <= kafkaMetadataServiceDiscoveryFailureThreshold) { + logger.warn("Swallowing metadata service error", t); + // reuse state + return latestKafkaStreams; + } else { + throw new RuntimeException( + "Fetching subscribed Kafka streams failed and no metadata to fallback", t); + } + } else { + // reset count in absence of failure + kafkaMetadataServiceDiscoveryFailureCount = 0; + return fetchedKafkaStreams; + } + } + + /** NOTE: Must run on coordinator thread. */ + private void sendMetadataUpdateEventToAvailableReaders() { + for (int readerId : enumContext.registeredReaders().keySet()) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", readerId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(readerId, metadataUpdateEvent); + } + } + + /** + * Initialize KafkaEnumerators, maybe with the topic partitions that are already assigned to by + * readers, to avoid duplicate re-assignment of splits. This is especially important in the + * restart mechanism when duplicate split assignment can cause undesired starting offsets (e.g. + * not assigning to the offsets prior to reader restart). Split offset resolution is mostly + * managed by the readers. + * + *

NOTE: Must run on coordinator thread + */ + private KafkaSourceEnumerator createEnumeratorWithAssignedTopicPartitions( + String kafkaClusterId, + Set topics, + KafkaSourceEnumState kafkaSourceEnumState, + Properties fetchedProperties) { + final Runnable signalNoMoreSplitsCallback; + if (Boundedness.BOUNDED.equals(boundedness)) { + signalNoMoreSplitsCallback = this::handleNoMoreSplits; + } else { + signalNoMoreSplitsCallback = null; + } + + StoppableKafkaEnumContextProxy context = + stoppableKafkaEnumContextProxyFactory.create( + enumContext, + kafkaClusterId, + kafkaMetadataService, + signalNoMoreSplitsCallback); + + Properties consumerProps = new Properties(); + KafkaPropertiesUtil.copyProperties(fetchedProperties, consumerProps); + KafkaPropertiesUtil.copyProperties(properties, consumerProps); + KafkaPropertiesUtil.setClientIdPrefix(consumerProps, kafkaClusterId); + + KafkaSourceEnumerator enumerator = + new KafkaSourceEnumerator( + KafkaSubscriber.getTopicListSubscriber(new ArrayList<>(topics)), + startingOffsetsInitializer, + stoppingOffsetInitializer, + consumerProps, + context, + boundedness, + kafkaSourceEnumState); + + clusterEnumContextMap.put(kafkaClusterId, context); + clusterEnumeratorMap.put(kafkaClusterId, enumerator); + + return enumerator; + } + + private void startAllEnumerators() { + for (String kafkaClusterId : latestClusterTopicsMap.keySet()) { + try { + // starts enumerators and handles split discovery and assignment + clusterEnumeratorMap.get(kafkaClusterId).start(); + } catch (KafkaException e) { + if (kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new RuntimeException( + String.format("Failed to create enumerator for %s", kafkaClusterId), e); + } else { + logger.info( + "Found inactive cluster {} while initializing, removing enumerator", + kafkaClusterId, + e); + try { + clusterEnumContextMap.remove(kafkaClusterId).close(); + clusterEnumeratorMap.remove(kafkaClusterId).close(); + } catch (Exception ex) { + // closing enumerator throws an exception, let error propagate and restart + // the job + throw new RuntimeException( + "Failed to close enum context for " + kafkaClusterId, ex); + } + } + } + } + } + + private void closeAllEnumeratorsAndContexts() { + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> { + try { + clusterEnumContextMap.get(cluster).close(); + subEnumerator.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + clusterEnumContextMap.clear(); + clusterEnumeratorMap.clear(); + } + + /** + * Multi cluster Kafka source readers will not request splits. Splits will be pushed to them, + * similarly for the sub enumerators. + */ + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + throw new UnsupportedOperationException("Kafka enumerators only assign splits to readers."); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + logger.debug("Adding splits back for {}", subtaskId); + // separate splits by cluster + ArrayListMultimap kafkaPartitionSplits = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + kafkaPartitionSplits.put(split.getKafkaClusterId(), split.getKafkaPartitionSplit()); + } + + // add splits back and assign pending splits for all enumerators + for (String kafkaClusterId : kafkaPartitionSplits.keySet()) { + if (clusterEnumeratorMap.containsKey(kafkaClusterId)) { + clusterEnumeratorMap + .get(kafkaClusterId) + .addSplitsBack(kafkaPartitionSplits.get(kafkaClusterId), subtaskId); + } else { + logger.warn( + "Split refers to inactive cluster {} with current clusters being {}", + kafkaClusterId, + clusterEnumeratorMap.keySet()); + } + } + + handleNoMoreSplits(); + } + + /** NOTE: this happens at startup and failover. */ + @Override + public void addReader(int subtaskId) { + logger.debug("Adding reader {}", subtaskId); + // assign pending splits from the sub enumerator + clusterEnumeratorMap.forEach( + (cluster, subEnumerator) -> subEnumerator.addReader(subtaskId)); + handleNoMoreSplits(); + } + + /** + * Besides for checkpointing, this method is used in the restart sequence to retain the relevant + * assigned splits so that there is no reader duplicate split assignment. See {@link + * #createEnumeratorWithAssignedTopicPartitions(String, Set, KafkaSourceEnumState, Properties)}} + */ + @Override + public DynamicKafkaSourceEnumState snapshotState(long checkpointId) throws Exception { + Map subEnumeratorStateByCluster = new HashMap<>(); + + // populate map for all assigned splits + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + subEnumeratorStateByCluster.put( + clusterEnumerator.getKey(), + clusterEnumerator.getValue().snapshotState(checkpointId)); + } + + return new DynamicKafkaSourceEnumState(latestKafkaStreams, subEnumeratorStateByCluster); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof GetMetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + if (enumContext.registeredReaders().containsKey(subtaskId)) { + MetadataUpdateEvent metadataUpdateEvent = new MetadataUpdateEvent(latestKafkaStreams); + logger.info("sending metadata update to reader {}: {}", subtaskId, metadataUpdateEvent); + enumContext.sendEventToSourceReader(subtaskId, metadataUpdateEvent); + } else { + logger.warn("Got get metadata update but subtask was unavailable"); + } + } + + @Override + public void close() throws IOException { + try { + // close contexts first since they may have running tasks + for (StoppableKafkaEnumContextProxy subEnumContext : clusterEnumContextMap.values()) { + subEnumContext.close(); + } + + for (Entry> + clusterEnumerator : clusterEnumeratorMap.entrySet()) { + clusterEnumerator.getValue().close(); + } + + kafkaMetadataService.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java new file mode 100644 index 000000000..752a5d6b2 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxy.java @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; + +import org.apache.kafka.common.KafkaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +/** + * A proxy enumerator context that supports life cycle management of underlying threads related to a + * sub {@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator}. This is + * motivated by the need to cancel the periodic partition discovery in scheduled tasks when sub + * Kafka Enumerators are restarted. The worker thread pool in {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext} should not contain tasks of + * inactive KafkaSourceEnumerators, after source restart. + * + *

Due to the inability to cancel scheduled tasks from {@link + * org.apache.flink.runtime.source.coordinator.SourceCoordinatorContext}, this enumerator context + * will safely catch exceptions during enumerator restart and use a closeable proxy scheduler to + * invoke tasks on the coordinator main thread to maintain the single threaded property. + */ +@Internal +public class StoppableKafkaEnumContextProxy + implements SplitEnumeratorContext, AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(StoppableKafkaEnumContextProxy.class); + + private final String kafkaClusterId; + private final KafkaMetadataService kafkaMetadataService; + private final SplitEnumeratorContext enumContext; + private final ScheduledExecutorService subEnumeratorWorker; + private final Runnable signalNoMoreSplitsCallback; + private boolean noMoreSplits = false; + private volatile boolean isClosing; + + /** + * Constructor for the enumerator context. + * + * @param kafkaClusterId The Kafka cluster id in order to maintain the mapping to the sub + * KafkaSourceEnumerator + * @param kafkaMetadataService the Kafka metadata service to facilitate error handling + * @param enumContext the underlying enumerator context + * @param signalNoMoreSplitsCallback the callback when signal no more splits is invoked + */ + public StoppableKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + SplitEnumeratorContext enumContext, + @Nullable Runnable signalNoMoreSplitsCallback) { + this.kafkaClusterId = kafkaClusterId; + this.kafkaMetadataService = kafkaMetadataService; + this.enumContext = enumContext; + this.subEnumeratorWorker = + Executors.newScheduledThreadPool( + 1, new ExecutorThreadFactory(kafkaClusterId + "-enum-worker")); + this.signalNoMoreSplitsCallback = signalNoMoreSplitsCallback; + this.isClosing = false; + } + + @Override + public SplitEnumeratorMetricGroup metricGroup() { + return enumContext.metricGroup(); + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + enumContext.sendEventToSourceReader(subtaskId, event); + } + + @Override + public int currentParallelism() { + return enumContext.currentParallelism(); + } + + @Override + public Map registeredReaders() { + return enumContext.registeredReaders(); + } + + /** Wrap splits with cluster metadata. */ + @Override + public void assignSplits(SplitsAssignment newSplitAssignments) { + if (logger.isInfoEnabled()) { + logger.info( + "Assigning {} splits for cluster {}: {}", + newSplitAssignments.assignment().values().stream() + .mapToLong(Collection::size) + .sum(), + kafkaClusterId, + newSplitAssignments); + } + + Map> readerToSplitsMap = new HashMap<>(); + newSplitAssignments + .assignment() + .forEach( + (subtask, splits) -> + readerToSplitsMap.put( + subtask, + splits.stream() + .map( + split -> + new DynamicKafkaSourceSplit( + kafkaClusterId, split)) + .collect(Collectors.toList()))); + + if (!readerToSplitsMap.isEmpty()) { + enumContext.assignSplits(new SplitsAssignment<>(readerToSplitsMap)); + } + } + + @Override + public void signalNoMoreSplits(int subtask) { + // There are no more splits for this cluster, but we need to wait until all clusters are + // finished with their respective split discoveries. In the Kafka Source, this is called in + // the coordinator thread, ensuring thread safety, for all source readers at the same time. + noMoreSplits = true; + if (signalNoMoreSplitsCallback != null) { + // Thread safe idempotent callback + signalNoMoreSplitsCallback.run(); + } + } + + /** Execute the one time callables in the coordinator. */ + @Override + public void callAsync(Callable callable, BiConsumer handler) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), wrapCallAsyncCallableHandler(handler)); + } + + /** + * Schedule task via internal thread pool to proxy task so that the task handler callback can + * execute in the single threaded source coordinator thread pool to avoid synchronization needs. + * + *

Having the scheduled task in the internal thread pool also allows us to cancel the task + * when the context needs to close due to dynamic enumerator restart. + * + *

In the case of KafkaEnumerator partition discovery, the callback modifies KafkaEnumerator + * object state. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + subEnumeratorWorker.scheduleAtFixedRate( + () -> callAsync(callable, handler), initialDelay, period, TimeUnit.MILLISECONDS); + } + + @Override + public void runInCoordinatorThread(Runnable runnable) { + enumContext.runInCoordinatorThread(runnable); + } + + public boolean isNoMoreSplits() { + return noMoreSplits; + } + + /** + * Note that we can't close the source coordinator here, because these contexts can be closed + * during metadata change when the coordinator still needs to continue to run. We can only close + * the coordinator context in Flink job shutdown, which Flink will do for us. That's why there + * is the complexity of the internal thread pools in this class. + * + *

TODO: Attach Flink JIRA ticket -- discuss with upstream how to cancel scheduled tasks + * belonging to enumerator. + */ + @Override + public void close() throws Exception { + logger.info("Closing enum context for {}", kafkaClusterId); + if (subEnumeratorWorker != null) { + // KafkaSubscriber worker thread will fail if admin client is closed in the middle. + // Swallow the error and set the context to closed state. + isClosing = true; + subEnumeratorWorker.shutdown(); + subEnumeratorWorker.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS); + } + } + + /** + * Wraps callable in call async executed in worker thread pool with exception propagation to + * optimize on doing IO in non-coordinator thread. + */ + protected Callable wrapCallAsyncCallable(Callable callable) { + return () -> { + try { + return callable.call(); + } catch (Exception e) { + if (isClosing) { + throw new HandledFlinkKafkaException(e, kafkaClusterId); + } + + Optional throwable = + ExceptionUtils.findThrowable(e, KafkaException.class); + // check if Kafka related and if Kafka cluster is inactive + if (throwable.isPresent() + && !kafkaMetadataService.isClusterActive(kafkaClusterId)) { + throw new HandledFlinkKafkaException(throwable.get(), kafkaClusterId); + } + + throw e; + } + }; + } + + /** + * Handle exception that is propagated by a callable, executed on coordinator thread. Depending + * on condition(s) the exception may be swallowed or forwarded. This is the Kafka topic + * partition discovery callable handler. + */ + protected BiConsumer wrapCallAsyncCallableHandler( + BiConsumer mainHandler) { + return (result, t) -> { + // check if exception is handled + Optional throwable = + ExceptionUtils.findThrowable(t, HandledFlinkKafkaException.class); + if (throwable.isPresent()) { + logger.warn("Swallowed handled exception for {}.", kafkaClusterId, throwable.get()); + return; + } + + // let the main handler deal with the potential exception + mainHandler.accept(result, t); + }; + } + + /** + * General exception to signal to internal exception handling mechanisms that a benign error + * occurred. + */ + @Internal + public static class HandledFlinkKafkaException extends RuntimeException { + private static final String ERROR_MESSAGE = "An error occurred with %s"; + + private final String kafkaClusterId; + + public HandledFlinkKafkaException(Throwable cause, String kafkaClusterId) { + super(cause); + this.kafkaClusterId = kafkaClusterId; + } + + public String getMessage() { + return String.format(ERROR_MESSAGE, kafkaClusterId); + } + } + + /** + * This factory exposes a way to override the {@link StoppableKafkaEnumContextProxy} used in the + * enumerator. This pluggable factory is extended in unit tests to facilitate invoking the + * periodic discovery loops on demand. + */ + @Internal + public interface StoppableKafkaEnumContextProxyFactory { + + StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + Runnable signalNoMoreSplitsCallback); + + static StoppableKafkaEnumContextProxyFactory getDefaultFactory() { + return (enumContext, + kafkaClusterId, + kafkaMetadataService, + signalNoMoreSplitsCallback) -> + new StoppableKafkaEnumContextProxy( + kafkaClusterId, + kafkaMetadataService, + enumContext, + signalNoMoreSplitsCallback); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java new file mode 100644 index 000000000..01dabf59c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSetSubscriber.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.util.HashSet; +import java.util.Set; + +/** Subscribe to streams based on the set of ids. */ +@Internal +public class KafkaStreamSetSubscriber implements KafkaStreamSubscriber { + + private final Set streamIds; + + public KafkaStreamSetSubscriber(Set streamIds) { + this.streamIds = streamIds; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + return new HashSet<>(kafkaMetadataService.describeStreams(streamIds).values()); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java new file mode 100644 index 000000000..bbd4e7f6c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/KafkaStreamSubscriber.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import java.io.Serializable; +import java.util.Set; + +/** + * The subscriber interacts with {@link KafkaMetadataService} to find which Kafka streams the source + * will subscribe to. + */ +@Experimental +public interface KafkaStreamSubscriber extends Serializable { + + /** + * Get the subscribed {@link KafkaStream}s. + * + * @param kafkaMetadataService the {@link KafkaMetadataService}. + * @return the subscribed {@link KafkaStream}s. + */ + Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java new file mode 100644 index 000000000..d95f7e0f0 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/subscriber/StreamPatternSubscriber.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableSet; + +import java.util.Set; +import java.util.regex.Pattern; + +/** To subscribe to streams based on a pattern. */ +@Internal +public class StreamPatternSubscriber implements KafkaStreamSubscriber { + + private final Pattern streamPattern; + + public StreamPatternSubscriber(Pattern streamPattern) { + this.streamPattern = streamPattern; + } + + @Override + public Set getSubscribedStreams(KafkaMetadataService kafkaMetadataService) { + Set allStreams = kafkaMetadataService.getAllStreams(); + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (KafkaStream kafkaStream : allStreams) { + String streamId = kafkaStream.getStreamId(); + if (streamPattern.matcher(streamId).find()) { + builder.add(kafkaStream); + } + } + + return builder.build(); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java new file mode 100644 index 000000000..451a05b3a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroup.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.runtime.metrics.MetricNames; + +import java.util.Map; + +/** + * A custom proxy metric group in order to group {@link + * org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics} by Kafka cluster. + * + *

Reports pending records per cluster under DynamicKafkaSource metric group, motivated by + * standardized connector metrics: + * https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics. + */ +@Internal +public class KafkaClusterMetricGroup implements SourceReaderMetricGroup { + + public static final String DYNAMIC_KAFKA_SOURCE_METRIC_GROUP = "DynamicKafkaSource"; + public static final String KAFKA_CLUSTER_GROUP_NAME = "kafkaCluster"; + + private final MetricGroup internalClusterSpecificMetricGroup; + private final OperatorIOMetricGroup delegateIOMetricGroup; + + public KafkaClusterMetricGroup( + MetricGroup dynamicKafkaSourceMetricGroup, + SourceReaderMetricGroup delegateSourceReaderMetricGroup, + String kafkaClusterId) { + this.internalClusterSpecificMetricGroup = + dynamicKafkaSourceMetricGroup.addGroup(KAFKA_CLUSTER_GROUP_NAME, kafkaClusterId); + this.delegateIOMetricGroup = delegateSourceReaderMetricGroup.getIOMetricGroup(); + } + + MetricGroup getInternalClusterSpecificMetricGroup() { + return internalClusterSpecificMetricGroup; + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingBytesGauge(Gauge gauge) { + throw new UnsupportedOperationException( + "This is not invoked/supported by KafkaSourceReader as of Flink 1.14."); + } + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) { + gauge(MetricNames.PENDING_RECORDS, pendingRecordsGauge); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return delegateIOMetricGroup; + } + + // --------------- MetricGroup methods --------------- + + @Override + public Counter counter(String name) { + return internalClusterSpecificMetricGroup.counter(name); + } + + @Override + public C counter(String name, C counter) { + return internalClusterSpecificMetricGroup.counter(name, counter); + } + + @Override + public > G gauge(String name, G gauge) { + return internalClusterSpecificMetricGroup.gauge(name, gauge); + } + + @Override + public H histogram(String name, H histogram) { + return internalClusterSpecificMetricGroup.histogram(name, histogram); + } + + @Override + public M meter(String name, M meter) { + return internalClusterSpecificMetricGroup.meter(name, meter); + } + + @Override + public MetricGroup addGroup(String name) { + return internalClusterSpecificMetricGroup.addGroup(name); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return internalClusterSpecificMetricGroup.addGroup(key, value); + } + + @Override + public String[] getScopeComponents() { + return internalClusterSpecificMetricGroup.getScopeComponents(); + } + + @Override + public Map getAllVariables() { + return internalClusterSpecificMetricGroup.getAllVariables(); + } + + @Override + public String getMetricIdentifier(String metricName) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName); + } + + @Override + public String getMetricIdentifier(String metricName, CharacterFilter filter) { + return internalClusterSpecificMetricGroup.getMetricIdentifier(metricName, filter); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java new file mode 100644 index 000000000..0659f528e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupManager.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** Manages metric groups for each cluster. */ +@Internal +public class KafkaClusterMetricGroupManager implements AutoCloseable { + private static final Logger logger = + LoggerFactory.getLogger(KafkaClusterMetricGroupManager.class); + private final Map metricGroups; + + public KafkaClusterMetricGroupManager() { + this.metricGroups = new HashMap<>(); + } + + public void register(String kafkaClusterId, KafkaClusterMetricGroup kafkaClusterMetricGroup) { + if (kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup() + instanceof AbstractMetricGroup) { + metricGroups.put( + kafkaClusterId, + (AbstractMetricGroup) + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup()); + } else { + logger.warn( + "MetricGroup {} is an instance of {}, which is not supported. Please use an implementation of AbstractMetricGroup.", + kafkaClusterMetricGroup.getInternalClusterSpecificMetricGroup(), + kafkaClusterMetricGroup + .getInternalClusterSpecificMetricGroup() + .getClass() + .getSimpleName()); + } + } + + public void close(String kafkaClusterId) { + AbstractMetricGroup metricGroup = metricGroups.remove(kafkaClusterId); + if (metricGroup != null) { + metricGroup.close(); + } else { + logger.warn( + "Tried to close metric group for {} but it is not registered for lifecycle management", + kafkaClusterId); + } + } + + @Override + public void close() throws Exception { + for (AbstractMetricGroup metricGroup : metricGroups.values()) { + metricGroup.close(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java new file mode 100644 index 000000000..8220ea14c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReader.java @@ -0,0 +1,549 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup; +import org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroupManager; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; +import org.apache.flink.connector.kafka.source.reader.KafkaSourceReader; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.runtime.io.MultipleFuturesAvailabilityHelper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.UserCodeClassLoader; + +import com.google.common.collect.ArrayListMultimap; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Properties; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * Manages state about underlying {@link KafkaSourceReader} to collect records and commit offsets + * from multiple Kafka clusters. This reader also handles changes to Kafka topology by reacting to + * restart sequence initiated by the enumerator and suspending inconsistent sub readers. + * + *

First, in the restart sequence, we will receive the {@link MetadataUpdateEvent} from the + * enumerator, stop all KafkaSourceReaders, and retain the relevant splits. Second, enumerator will + * send all new splits that readers should work on (old splits will not be sent again). + */ +@Internal +public class DynamicKafkaSourceReader implements SourceReader { + private static final Logger logger = LoggerFactory.getLogger(DynamicKafkaSourceReader.class); + private final KafkaRecordDeserializationSchema deserializationSchema; + private final Properties properties; + private final MetricGroup dynamicKafkaSourceMetricGroup; + private final Gauge kafkaClusterCount; + private final SourceReaderContext readerContext; + private final KafkaClusterMetricGroupManager kafkaClusterMetricGroupManager; + + // needs have a strict ordering for readers to guarantee availability future consistency + private final NavigableMap> clusterReaderMap; + private final Map clustersProperties; + private final List pendingSplits; + + private MultipleFuturesAvailabilityHelper availabilityHelper; + private boolean isActivelyConsumingSplits; + private boolean isNoMoreSplits; + private AtomicBoolean restartingReaders; + + public DynamicKafkaSourceReader( + SourceReaderContext readerContext, + KafkaRecordDeserializationSchema deserializationSchema, + Properties properties) { + this.readerContext = readerContext; + this.clusterReaderMap = new TreeMap<>(); + this.deserializationSchema = deserializationSchema; + this.properties = properties; + this.kafkaClusterCount = clusterReaderMap::size; + this.dynamicKafkaSourceMetricGroup = + readerContext + .metricGroup() + .addGroup(KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + this.kafkaClusterMetricGroupManager = new KafkaClusterMetricGroupManager(); + this.pendingSplits = new ArrayList<>(); + this.availabilityHelper = new MultipleFuturesAvailabilityHelper(0); + this.isNoMoreSplits = false; + this.isActivelyConsumingSplits = false; + this.restartingReaders = new AtomicBoolean(); + this.clustersProperties = new HashMap<>(); + } + + /** + * This is invoked first only in reader startup without state. In stateful startup, splits are + * added before this method is invoked. + */ + @Override + public void start() { + logger.trace("Starting reader for subtask index={}", readerContext.getIndexOfSubtask()); + // metrics cannot be registered in the enumerator + readerContext.metricGroup().gauge("kafkaClusterCount", kafkaClusterCount); + readerContext.sendSourceEventToCoordinator(new GetMetadataUpdateEvent()); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + // at startup, do not return end of input if metadata event has not been received + if (clusterReaderMap.isEmpty()) { + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + if (restartingReaders.get()) { + logger.debug("Poll next invoked while restarting readers"); + return logAndReturnInputStatus(InputStatus.NOTHING_AVAILABLE); + } + + boolean isMoreAvailable = false; + boolean isNothingAvailable = false; + + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + InputStatus inputStatus = clusterReader.getValue().pollNext(readerOutput); + switch (inputStatus) { + case MORE_AVAILABLE: + isMoreAvailable = true; + break; + case NOTHING_AVAILABLE: + isNothingAvailable = true; + break; + } + } + + return logAndReturnInputStatus(consolidateInputStatus(isMoreAvailable, isNothingAvailable)); + } + + private InputStatus consolidateInputStatus( + boolean atLeastOneMoreAvailable, boolean atLeastOneNothingAvailable) { + final InputStatus inputStatus; + if (atLeastOneMoreAvailable) { + inputStatus = InputStatus.MORE_AVAILABLE; + } else if (atLeastOneNothingAvailable) { + inputStatus = InputStatus.NOTHING_AVAILABLE; + } else { + inputStatus = InputStatus.END_OF_INPUT; + } + return inputStatus; + } + + // we also need to filter splits at startup in case checkpoint is not consistent bwtn enumerator + // and reader + @Override + public void addSplits(List splits) { + logger.info("Adding splits to reader {}: {}", readerContext.getIndexOfSubtask(), splits); + // at startup, don't add splits until we get confirmation from enumerator of the current + // metadata + if (!isActivelyConsumingSplits) { + pendingSplits.addAll(splits); + return; + } + + ArrayListMultimap clusterSplitsMap = + ArrayListMultimap.create(); + for (DynamicKafkaSourceSplit split : splits) { + clusterSplitsMap.put(split.getKafkaClusterId(), split); + } + + Set kafkaClusterIds = clusterSplitsMap.keySet(); + + boolean newCluster = false; + for (String kafkaClusterId : kafkaClusterIds) { + // if a reader corresponding to the split doesn't exist, create it + // it is possible that the splits come before the source event + if (!clusterReaderMap.containsKey(kafkaClusterId)) { + try { + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + kafkaSourceReader.start(); + newCluster = true; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // add splits + KafkaSourceReader reader = clusterReaderMap.get(kafkaClusterId); + reader.addSplits(clusterSplitsMap.get(kafkaClusterId)); + } + + // reset the availability future to also depend on the new sub readers + if (newCluster) { + completeAndResetAvailabilityHelper(); + } + } + + /** + * Duplicate source events are handled with idempotency. No metadata change means we simply skip + * the restart logic. + */ + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + Preconditions.checkArgument( + sourceEvent instanceof MetadataUpdateEvent, + "Received invalid source event: " + sourceEvent); + + logger.info( + "Received source event {}: subtask={}", + sourceEvent, + readerContext.getIndexOfSubtask()); + Set newKafkaStreams = ((MetadataUpdateEvent) sourceEvent).getKafkaStreams(); + Map> newClustersAndTopics = new HashMap<>(); + Map newClustersProperties = new HashMap<>(); + for (KafkaStream kafkaStream : newKafkaStreams) { + for (Map.Entry clusterMetadataMapEntry : + kafkaStream.getClusterMetadataMap().entrySet()) { + newClustersAndTopics + .computeIfAbsent( + clusterMetadataMapEntry.getKey(), (unused) -> new HashSet<>()) + .addAll(clusterMetadataMapEntry.getValue().getTopics()); + + newClustersProperties.put( + clusterMetadataMapEntry.getKey(), + clusterMetadataMapEntry.getValue().getProperties()); + } + } + + // filter current splits with the metadata update + List currentSplitState = snapshotStateFromAllReaders(-1); + logger.info( + "Snapshotting split state for reader {}: {}", + readerContext.getIndexOfSubtask(), + currentSplitState); + Map> currentMetadataFromState = new HashMap<>(); + Map> filteredNewClusterSplitStateMap = new HashMap<>(); + + // the data structures above + for (DynamicKafkaSourceSplit split : currentSplitState) { + currentMetadataFromState + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new HashSet<>()) + .add(split.getKafkaPartitionSplit().getTopic()); + // check if cluster topic exists in the metadata update + if (newClustersAndTopics.containsKey(split.getKafkaClusterId()) + && newClustersAndTopics + .get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic())) { + filteredNewClusterSplitStateMap + .computeIfAbsent(split.getKafkaClusterId(), (ignore) -> new ArrayList<>()) + .add(split); + } else { + logger.info("Skipping outdated split due to metadata changes: {}", split); + } + } + + // only restart if there was metadata change to handle duplicate MetadataUpdateEvent from + // enumerator. We can possibly only restart the readers whose metadata has changed but that + // comes at the cost of complexity and it is an optimization for a corner case. We can + // revisit if necessary. + if (!newClustersAndTopics.equals(currentMetadataFromState)) { + restartingReaders.set(true); + closeAllReadersAndClearState(); + + clustersProperties.putAll(newClustersProperties); + for (String kafkaClusterId : newClustersAndTopics.keySet()) { + try { + // restart kafka source readers with the relevant state + KafkaSourceReader kafkaSourceReader = createReader(kafkaClusterId); + clusterReaderMap.put(kafkaClusterId, kafkaSourceReader); + if (filteredNewClusterSplitStateMap.containsKey(kafkaClusterId)) { + kafkaSourceReader.addSplits( + filteredNewClusterSplitStateMap.get(kafkaClusterId)); + } + kafkaSourceReader.start(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + // reset the availability future to also depend on the new sub readers + completeAndResetAvailabilityHelper(); + } else { + // update properties even on no metadata change + clustersProperties.clear(); + clustersProperties.putAll(newClustersProperties); + } + + // finally mark the reader as active, if not already and add pending splits + if (!isActivelyConsumingSplits) { + isActivelyConsumingSplits = true; + } + + if (!pendingSplits.isEmpty()) { + List validPendingSplits = + pendingSplits.stream() + // Pending splits is used to cache splits at startup, before metadata + // update event arrives. Splits in state could be old and it's possible + // to not have another metadata update event, so need to filter the + // splits at this point. + .filter( + pendingSplit -> { + boolean splitValid = + isSplitForActiveClusters( + pendingSplit, newClustersAndTopics); + if (!splitValid) { + logger.info( + "Removing invalid split for reader: {}", + pendingSplit); + } + return splitValid; + }) + .collect(Collectors.toList()); + + addSplits(validPendingSplits); + pendingSplits.clear(); + if (isNoMoreSplits) { + notifyNoMoreSplits(); + } + } + } + + private static boolean isSplitForActiveClusters( + DynamicKafkaSourceSplit split, Map> metadata) { + return metadata.containsKey(split.getKafkaClusterId()) + && metadata.get(split.getKafkaClusterId()) + .contains(split.getKafkaPartitionSplit().getTopic()); + } + + @Override + public List snapshotState(long checkpointId) { + List splits = snapshotStateFromAllReaders(checkpointId); + + // pending splits should be typically empty, since we do not add splits to pending splits if + // reader has started + splits.addAll(pendingSplits); + return splits; + } + + private List snapshotStateFromAllReaders(long checkpointId) { + List splits = new ArrayList<>(); + for (Map.Entry> clusterReader : clusterReaderMap.entrySet()) { + clusterReader + .getValue() + .snapshotState(checkpointId) + .forEach( + kafkaPartitionSplit -> + splits.add( + new DynamicKafkaSourceSplit( + clusterReader.getKey(), kafkaPartitionSplit))); + } + + return splits; + } + + @Override + public CompletableFuture isAvailable() { + availabilityHelper.resetToUnAvailable(); + syncAvailabilityHelperWithReaders(); + return (CompletableFuture) availabilityHelper.getAvailableFuture(); + } + + @Override + public void notifyNoMoreSplits() { + logger.info("notify no more splits for reader {}", readerContext.getIndexOfSubtask()); + if (pendingSplits.isEmpty()) { + clusterReaderMap.values().forEach(KafkaSourceReader::notifyNoMoreSplits); + } + + isNoMoreSplits = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + logger.debug("Notify checkpoint complete for {}", clusterReaderMap.keySet()); + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.notifyCheckpointComplete(checkpointId); + } + } + + @Override + public void close() throws Exception { + for (KafkaSourceReader subReader : clusterReaderMap.values()) { + subReader.close(); + } + kafkaClusterMetricGroupManager.close(); + } + + private KafkaSourceReader createReader(String kafkaClusterId) throws Exception { + FutureCompletingBlockingQueue>> + elementsQueue = new FutureCompletingBlockingQueue<>(); + Properties readerSpecificProperties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, readerSpecificProperties); + KafkaPropertiesUtil.copyProperties( + Preconditions.checkNotNull( + clustersProperties.get(kafkaClusterId), + "Properties for cluster %s is not found. Current Kafka cluster ids: %s", + kafkaClusterId, + clustersProperties.keySet()), + readerSpecificProperties); + KafkaPropertiesUtil.setClientIdPrefix(readerSpecificProperties, kafkaClusterId); + + // layer a kafka cluster group to distinguish metrics by cluster + KafkaClusterMetricGroup kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + dynamicKafkaSourceMetricGroup, readerContext.metricGroup(), kafkaClusterId); + kafkaClusterMetricGroupManager.register(kafkaClusterId, kafkaClusterMetricGroup); + KafkaSourceReaderMetrics kafkaSourceReaderMetrics = + new KafkaSourceReaderMetrics(kafkaClusterMetricGroup); + + deserializationSchema.open( + new DeserializationSchema.InitializationContext() { + @Override + public MetricGroup getMetricGroup() { + // adding kafkaClusterMetricGroup instead of the sourceReaderMetricGroup + // since there could be metric collision, so `kafkaCluster` group is + // necessary to + // distinguish between instances of this metric + return kafkaClusterMetricGroup.addGroup("deserializer"); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + }); + + KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); + return new KafkaSourceReader<>( + elementsQueue, + new KafkaSourceFetcherManager( + elementsQueue, + () -> + new KafkaPartitionSplitReaderWrapper( + readerSpecificProperties, + readerContext, + kafkaSourceReaderMetrics, + kafkaClusterId), + (ignore) -> {}), + recordEmitter, + toConfiguration(readerSpecificProperties), + readerContext, + kafkaSourceReaderMetrics); + } + + /** + * In metadata change, we need to reset the availability helper since the number of Kafka source + * readers could have changed. + */ + private void completeAndResetAvailabilityHelper() { + CompletableFuture cachedPreviousFuture = availabilityHelper.getAvailableFuture(); + availabilityHelper = new MultipleFuturesAvailabilityHelper(clusterReaderMap.size()); + syncAvailabilityHelperWithReaders(); + + // We cannot immediately complete the previous future here. We must complete it only when + // the new readers have finished handling the split assignment. Completing the future too + // early can cause WakeupException (implicitly woken up by invocation to pollNext()) if the + // reader has not finished resolving the positions of the splits, as seen in flaky unit test + // errors. There is no error handling for WakeupException in SplitReader's + // handleSplitChanges. + availabilityHelper + .getAvailableFuture() + .whenComplete( + (ignore, t) -> { + restartingReaders.set(false); + cachedPreviousFuture.complete(null); + }); + } + + private void syncAvailabilityHelperWithReaders() { + int i = 0; + for (String kafkaClusterId : clusterReaderMap.navigableKeySet()) { + availabilityHelper.anyOf(i, clusterReaderMap.get(kafkaClusterId).isAvailable()); + i++; + } + } + + private void closeAllReadersAndClearState() { + for (Map.Entry> entry : clusterReaderMap.entrySet()) { + try { + logger.info( + "Closing sub reader in reader {} for cluster: {}", + readerContext.getIndexOfSubtask(), + entry.getKey()); + entry.getValue().close(); + kafkaClusterMetricGroupManager.close(entry.getKey()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + clusterReaderMap.clear(); + clustersProperties.clear(); + } + + static Configuration toConfiguration(Properties props) { + Configuration config = new Configuration(); + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); + return config; + } + + private InputStatus logAndReturnInputStatus(InputStatus inputStatus) { + if (InputStatus.END_OF_INPUT.equals(inputStatus)) { + logger.info( + "inputStatus={}, subtaskIndex={}", + inputStatus, + readerContext.getIndexOfSubtask()); + } else { + logger.trace( + "inputStatus={}, subtaskIndex={}", + inputStatus, + readerContext.getIndexOfSubtask()); + } + + return inputStatus; + } + + @VisibleForTesting + public MultipleFuturesAvailabilityHelper getAvailabilityHelper() { + return availabilityHelper; + } + + @VisibleForTesting + public boolean isActivelyConsumingSplits() { + return isActivelyConsumingSplits; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java new file mode 100644 index 000000000..cc250eace --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/reader/KafkaPartitionSplitReaderWrapper.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** This extends to Kafka Partition Split Reader to wrap split ids with the cluster name. */ +@Internal +public class KafkaPartitionSplitReaderWrapper extends KafkaPartitionSplitReader + implements AutoCloseable { + private final String kafkaClusterId; + + public KafkaPartitionSplitReaderWrapper( + Properties props, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics, + String kafkaClusterId) { + super(props, context, kafkaSourceReaderMetrics); + this.kafkaClusterId = kafkaClusterId; + } + + @Override + public RecordsWithSplitIds> fetch() throws IOException { + return new WrappedRecordsWithSplitIds(super.fetch(), kafkaClusterId); + } + + private static final class WrappedRecordsWithSplitIds + implements RecordsWithSplitIds> { + + private final RecordsWithSplitIds> delegate; + private final String kafkaClusterId; + + public WrappedRecordsWithSplitIds( + RecordsWithSplitIds> delegate, + String kafkaClusterId) { + this.delegate = delegate; + this.kafkaClusterId = kafkaClusterId; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = delegate.nextSplit(); + if (nextSplit == null) { + return nextSplit; + } else { + return kafkaClusterId + "-" + nextSplit; + } + } + + @Nullable + @Override + public ConsumerRecord nextRecordFromSplit() { + return delegate.nextRecordFromSplit(); + } + + @Override + public Set finishedSplits() { + return delegate.finishedSplits().stream() + .map(finishedSplit -> kafkaClusterId + "-" + finishedSplit) + .collect(Collectors.toSet()); + } + + @Override + public void recycle() { + delegate.recycle(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java new file mode 100644 index 000000000..25ef25b99 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplit.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import com.google.common.base.MoreObjects; + +import java.util.Objects; + +/** Split that wraps {@link KafkaPartitionSplit} with Kafka cluster information. */ +@Internal +public class DynamicKafkaSourceSplit extends KafkaPartitionSplit { + + private final String kafkaClusterId; + private final KafkaPartitionSplit kafkaPartitionSplit; + + public DynamicKafkaSourceSplit(String kafkaClusterId, KafkaPartitionSplit kafkaPartitionSplit) { + super( + kafkaPartitionSplit.getTopicPartition(), + kafkaPartitionSplit.getStartingOffset(), + kafkaPartitionSplit.getStoppingOffset().orElse(NO_STOPPING_OFFSET)); + this.kafkaClusterId = kafkaClusterId; + this.kafkaPartitionSplit = kafkaPartitionSplit; + } + + @Override + public String splitId() { + return kafkaClusterId + "-" + kafkaPartitionSplit.splitId(); + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaPartitionSplit getKafkaPartitionSplit() { + return kafkaPartitionSplit; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaPartitionSplit", kafkaPartitionSplit) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + DynamicKafkaSourceSplit that = (DynamicKafkaSourceSplit) o; + return Objects.equals(kafkaClusterId, that.kafkaClusterId) + && Objects.equals(kafkaPartitionSplit, that.kafkaPartitionSplit); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), kafkaClusterId, kafkaPartitionSplit); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java new file mode 100644 index 000000000..852894f8b --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializer.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import com.google.common.io.ByteStreams; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** (De)serializes the {@link DynamicKafkaSourceSplit}. */ +@Internal +public class DynamicKafkaSourceSplitSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION_1 = 1; + + private final KafkaPartitionSplitSerializer kafkaPartitionSplitSerializer; + + public DynamicKafkaSourceSplitSerializer() { + this.kafkaPartitionSplitSerializer = new KafkaPartitionSplitSerializer(); + } + + @Override + public int getVersion() { + return VERSION_1; + } + + @Override + public byte[] serialize(DynamicKafkaSourceSplit split) throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeUTF(split.getKafkaClusterId()); + out.writeInt(kafkaPartitionSplitSerializer.getVersion()); + out.write(kafkaPartitionSplitSerializer.serialize(split.getKafkaPartitionSplit())); + out.flush(); + return baos.toByteArray(); + } + } + + @Override + public DynamicKafkaSourceSplit deserialize(int version, byte[] serialized) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + String kafkaClusterId = in.readUTF(); + int kafkaPartitionSplitSerializerVersion = in.readInt(); + KafkaPartitionSplit kafkaPartitionSplit = + kafkaPartitionSplitSerializer.deserialize( + kafkaPartitionSplitSerializerVersion, ByteStreams.toByteArray(in)); + return new DynamicKafkaSourceSplit(kafkaClusterId, kafkaPartitionSplit); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java new file mode 100644 index 000000000..e1c682345 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetFacet.java @@ -0,0 +1,65 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.kafka.source.KafkaPropertiesUtil; + +import java.util.Objects; +import java.util.Properties; + +/** Default implementation of {@link KafkaDatasetFacet}. */ +@PublicEvolving +public class DefaultKafkaDatasetFacet implements KafkaDatasetFacet { + + public static final String KAFKA_FACET_NAME = "kafka"; + + private Properties properties; + + private final KafkaDatasetIdentifier topicIdentifier; + + public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier, Properties properties) { + this(topicIdentifier); + + this.properties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, this.properties); + } + + public DefaultKafkaDatasetFacet(KafkaDatasetIdentifier topicIdentifier) { + this.topicIdentifier = topicIdentifier; + } + + public void setProperties(Properties properties) { + this.properties = new Properties(); + KafkaPropertiesUtil.copyProperties(properties, this.properties); + } + + public Properties getProperties() { + return properties; + } + + public KafkaDatasetIdentifier getTopicIdentifier() { + return topicIdentifier; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultKafkaDatasetFacet that = (DefaultKafkaDatasetFacet) o; + return Objects.equals(properties, that.properties) + && Objects.equals(topicIdentifier, that.topicIdentifier); + } + + @Override + public int hashCode() { + return Objects.hash(properties, topicIdentifier); + } + + @Override + public String name() { + return KAFKA_FACET_NAME; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java new file mode 100644 index 000000000..cd97b7ff4 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultKafkaDatasetIdentifier.java @@ -0,0 +1,59 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; + +/** Default implementation of {@link KafkaDatasetIdentifier}. */ +@PublicEvolving +public class DefaultKafkaDatasetIdentifier implements KafkaDatasetIdentifier { + + @Nullable private final List topics; + @Nullable private final Pattern topicPattern; + + private DefaultKafkaDatasetIdentifier( + @Nullable List fixedTopics, @Nullable Pattern topicPattern) { + this.topics = fixedTopics; + this.topicPattern = topicPattern; + } + + public static DefaultKafkaDatasetIdentifier ofPattern(Pattern pattern) { + return new DefaultKafkaDatasetIdentifier(null, pattern); + } + + public static DefaultKafkaDatasetIdentifier ofTopics(List fixedTopics) { + return new DefaultKafkaDatasetIdentifier(fixedTopics, null); + } + + @Nullable + public List getTopics() { + return topics; + } + + @Nullable + public Pattern getTopicPattern() { + return topicPattern; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultKafkaDatasetIdentifier that = (DefaultKafkaDatasetIdentifier) o; + return Objects.equals(topics, that.topics) + && Objects.equals(topicPattern, that.topicPattern); + } + + @Override + public int hashCode() { + return Objects.hash(topics, topicPattern); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java new file mode 100644 index 000000000..d9475d77a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/DefaultTypeDatasetFacet.java @@ -0,0 +1,44 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; + +import java.util.Objects; + +/** Default implementation of {@link KafkaDatasetFacet}. */ +@PublicEvolving +public class DefaultTypeDatasetFacet implements TypeDatasetFacet { + + public static final String TYPE_FACET_NAME = "type"; + + private final TypeInformation typeInformation; + + public DefaultTypeDatasetFacet(TypeInformation typeInformation) { + this.typeInformation = typeInformation; + } + + public TypeInformation getTypeInformation() { + return typeInformation; + } + + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultTypeDatasetFacet that = (DefaultTypeDatasetFacet) o; + return Objects.equals(typeInformation, that.typeInformation); + } + + @Override + public int hashCode() { + return Objects.hash(typeInformation); + } + + @Override + public String name() { + return TYPE_FACET_NAME; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java new file mode 100644 index 000000000..c0d3d0b73 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacet.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; + +import java.util.Properties; + +/** Facet definition to contain all Kafka specific information on Kafka sources and sinks. */ +@PublicEvolving +public interface KafkaDatasetFacet extends LineageDatasetFacet { + Properties getProperties(); + + KafkaDatasetIdentifier getTopicIdentifier(); + + void setProperties(Properties properties); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java new file mode 100644 index 000000000..76fe41b82 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetFacetProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method to extract {@link KafkaDatasetFacet}. */ +@PublicEvolving +public interface KafkaDatasetFacetProvider { + + /** + * Returns a Kafka dataset facet or empty in case an implementing class is not able to identify + * a dataset. + */ + Optional getKafkaDatasetFacet(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java new file mode 100644 index 000000000..19f7082e2 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifier.java @@ -0,0 +1,30 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.regex.Pattern; + +/** Kafka dataset identifier which can contain either a list of topics or a topic pattern. */ +@PublicEvolving +public interface KafkaDatasetIdentifier { + @Nullable + List getTopics(); + + @Nullable + Pattern getTopicPattern(); + + /** + * Assigns lineage dataset's name which is topic pattern if it is present or comma separated + * list of topics. + */ + default String toLineageName() { + if (getTopicPattern() != null) { + return getTopicPattern().toString(); + } + return String.join(",", Objects.requireNonNull(getTopics())); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java new file mode 100644 index 000000000..1389fea58 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/KafkaDatasetIdentifierProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method which allows extracting topic identifier. */ +@PublicEvolving +public interface KafkaDatasetIdentifierProvider { + + /** + * Gets Kafka dataset identifier or empty in case a class implementing is not able to extract + * dataset identifier. + */ + Optional getDatasetIdentifier(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java new file mode 100644 index 000000000..086303e09 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/LineageUtil.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + * + */ + +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +/** Utility class with useful methods for managing lineage objects. */ +public class LineageUtil { + + private static final String KAFKA_DATASET_PREFIX = "kafka://"; + private static final String COMMA = ","; + private static final String SEMICOLON = ";"; + + public static LineageDataset datasetOf(String namespace, KafkaDatasetFacet kafkaDatasetFacet) { + return datasetOf(namespace, kafkaDatasetFacet, Collections.emptyList()); + } + + public static LineageDataset datasetOf( + String namespace, KafkaDatasetFacet kafkaDatasetFacet, TypeDatasetFacet typeFacet) { + return datasetOf(namespace, kafkaDatasetFacet, Collections.singletonList(typeFacet)); + } + + private static LineageDataset datasetOf( + String namespace, + KafkaDatasetFacet kafkaDatasetFacet, + List facets) { + return new LineageDataset() { + @Override + public String name() { + return kafkaDatasetFacet.getTopicIdentifier().toLineageName(); + } + + @Override + public String namespace() { + return namespace; + } + + @Override + public Map facets() { + Map facetMap = new HashMap<>(); + facetMap.put(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME, kafkaDatasetFacet); + facetMap.putAll( + facets.stream() + .collect( + Collectors.toMap(LineageDatasetFacet::name, item -> item))); + return facetMap; + } + }; + } + + public static String namespaceOf(Properties properties) { + String bootstrapServers = properties.getProperty("bootstrap.servers"); + + if (bootstrapServers == null) { + return KAFKA_DATASET_PREFIX; + } + + if (bootstrapServers.contains(COMMA)) { + bootstrapServers = bootstrapServers.split(COMMA)[0]; + } else if (bootstrapServers.contains(SEMICOLON)) { + bootstrapServers = bootstrapServers.split(SEMICOLON)[0]; + } + + return String.format(KAFKA_DATASET_PREFIX + bootstrapServers); + } + + public static SourceLineageVertex sourceLineageVertexOf(Collection datasets) { + return new SourceLineageVertex() { + @Override + public Boundedness boundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public List datasets() { + return datasets.stream().collect(Collectors.toList()); + } + }; + } + + public static LineageVertex lineageVertexOf(Collection datasets) { + return new LineageVertex() { + @Override + public List datasets() { + return datasets.stream().collect(Collectors.toList()); + } + }; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java new file mode 100644 index 000000000..1e64f5819 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacet.java @@ -0,0 +1,11 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; + +/** Facet definition to contain type information of source and sink. */ +@PublicEvolving +public interface TypeDatasetFacet extends LineageDatasetFacet { + TypeInformation getTypeInformation(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java new file mode 100644 index 000000000..016a1bb84 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/lineage/TypeDatasetFacetProvider.java @@ -0,0 +1,16 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** Contains method to extract {@link TypeDatasetFacet}. */ +@PublicEvolving +public interface TypeDatasetFacetProvider { + + /** + * Returns a type dataset facet or `Optional.empty` in case an implementing class is not able to + * resolve type. + */ + Optional getTypeDatasetFacet(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java new file mode 100644 index 000000000..4cf385fca --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriter.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ProducerPool; +import org.apache.flink.connector.kafka.sink.internal.ProducerPoolImpl; +import org.apache.flink.connector.kafka.sink.internal.ReadableBackchannel; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Properties; + +import static org.apache.flink.util.IOUtils.closeAll; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Exactly-once Kafka writer that writes records to Kafka in transactions. + * + * @param The type of the input elements. + */ +class ExactlyOnceKafkaWriter extends KafkaWriter { + private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceKafkaWriter.class); + private final String transactionalIdPrefix; + + private final KafkaWriterState kafkaWriterState; + private final Collection recoveredStates; + private final long restoredCheckpointId; + + /** + * The producer pool that manages all transactional producers. It keeps track of the producers + * that have been recycled as well as producers that are currently in use (potentially forwarded + * to committer). + */ + private final ProducerPool producerPool; + /** + * Backchannel used to communicate committed transactions from the committer to this writer. + * Establishing the channel happens during recovery. Thus, it is only safe to poll in checkpoint + * related methods. + */ + private final ReadableBackchannel backchannel; + + /** + * Constructor creating a kafka writer. + * + *

It will throw a {@link RuntimeException} if {@link + * KafkaRecordSerializationSchema#open(SerializationSchema.InitializationContext, + * KafkaRecordSerializationSchema.KafkaSinkContext)} fails. + * + * @param deliveryGuarantee the Sink's delivery guarantee + * @param kafkaProducerConfig the properties to configure the {@link FlinkKafkaInternalProducer} + * @param transactionalIdPrefix used to create the transactionalIds + * @param sinkInitContext context to provide information about the runtime environment + * @param recordSerializer serialize to transform the incoming records to {@link ProducerRecord} + * @param schemaContext context used to initialize the {@link KafkaRecordSerializationSchema} + * @param recoveredStates state from an previous execution which was covered + */ + ExactlyOnceKafkaWriter( + DeliveryGuarantee deliveryGuarantee, + Properties kafkaProducerConfig, + String transactionalIdPrefix, + Sink.InitContext sinkInitContext, + KafkaRecordSerializationSchema recordSerializer, + SerializationSchema.InitializationContext schemaContext, + Collection recoveredStates) { + super( + deliveryGuarantee, + kafkaProducerConfig, + sinkInitContext, + recordSerializer, + schemaContext); + this.transactionalIdPrefix = + checkNotNull(transactionalIdPrefix, "transactionalIdPrefix must not be null"); + + try { + recordSerializer.open(schemaContext, kafkaSinkContext); + } catch (Exception e) { + throw new FlinkRuntimeException("Cannot initialize schema.", e); + } + + this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix); + + this.recoveredStates = checkNotNull(recoveredStates, "recoveredStates"); + initFlinkMetrics(); + restoredCheckpointId = + sinkInitContext + .getRestoredCheckpointId() + .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); + int subtaskId = sinkInitContext.getTaskInfo().getIndexOfThisSubtask(); + this.producerPool = new ProducerPoolImpl(kafkaProducerConfig, this::initKafkaMetrics); + this.backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel( + subtaskId, + sinkInitContext.getTaskInfo().getAttemptNumber(), + transactionalIdPrefix); + } + + @Override + public void initialize() { + abortLingeringTransactions( + checkNotNull(recoveredStates, "recoveredStates"), restoredCheckpointId + 1); + this.currentProducer = startTransaction(restoredCheckpointId + 1); + } + + private FlinkKafkaInternalProducer startTransaction(long checkpointId) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer( + TransactionalIdFactory.buildTransactionalId( + transactionalIdPrefix, + kafkaSinkContext.getParallelInstanceId(), + checkpointId), + checkpointId); + producer.beginTransaction(); + return producer; + } + + @Override + public Collection prepareCommit() { + // only return a KafkaCommittable if the current transaction has been written some data + if (currentProducer.hasRecordsInTransaction()) { + KafkaCommittable committable = KafkaCommittable.of(currentProducer); + LOG.debug("Prepare {}.", committable); + return Collections.singletonList(committable); + } + + // otherwise, we recycle the producer (the pool will reset the transaction state) + producerPool.recycle(currentProducer); + return Collections.emptyList(); + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + // recycle committed producers + String finishedTransactionalId; + while ((finishedTransactionalId = backchannel.poll()) != null) { + producerPool.recycleByTransactionId(finishedTransactionalId); + } + currentProducer = startTransaction(checkpointId + 1); + return Collections.singletonList(kafkaWriterState); + } + + @Override + public void close() throws Exception { + closeAll( + this::abortCurrentProducer, + () -> closeAll(producerPool), + backchannel, + super::close); + } + + private void abortCurrentProducer() { + // only abort if the transaction is known to the broker (needs to have at least one record + // sent) + if (currentProducer.isInTransaction() && currentProducer.hasRecordsInTransaction()) { + try { + currentProducer.abortTransaction(); + } catch (ProducerFencedException e) { + LOG.debug( + "Producer {} fenced while aborting", currentProducer.getTransactionalId()); + } + } + } + + @VisibleForTesting + ProducerPool getProducerPool() { + return producerPool; + } + + @VisibleForTesting + public String getTransactionalIdPrefix() { + return transactionalIdPrefix; + } + + private void abortLingeringTransactions( + Collection recoveredStates, long startCheckpointId) { + List prefixesToAbort = new ArrayList<>(); + prefixesToAbort.add(transactionalIdPrefix); + + LOG.info( + "Aborting lingering transactions from previous execution. Recovered states: {}.", + recoveredStates); + final Optional lastStateOpt = recoveredStates.stream().findFirst(); + if (lastStateOpt.isPresent()) { + KafkaWriterState lastState = lastStateOpt.get(); + if (!lastState.getTransactionalIdPrefix().equals(transactionalIdPrefix)) { + prefixesToAbort.add(lastState.getTransactionalIdPrefix()); + LOG.warn( + "Transactional id prefix from previous execution {} has changed to {}.", + lastState.getTransactionalIdPrefix(), + transactionalIdPrefix); + } + } + + try (TransactionAborter transactionAborter = + new TransactionAborter( + kafkaSinkContext.getParallelInstanceId(), + kafkaSinkContext.getNumberOfParallelInstances(), + id -> producerPool.getTransactionalProducer(id, startCheckpointId), + producerPool::recycle)) { + transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java new file mode 100644 index 000000000..2c0c080b8 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/HeaderProvider.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; + +import java.io.Serializable; + +/** Creates an {@link Iterable} of {@link Header}s from the input element. */ +@PublicEvolving +public interface HeaderProvider extends Serializable { + Headers getHeaders(IN input); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java index 72a3281e6..1ce3b6bb3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommittable.java @@ -17,42 +17,43 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; + import javax.annotation.Nullable; import java.util.Objects; import java.util.Optional; -import java.util.function.Consumer; /** * This class holds the necessary information to construct a new {@link FlinkKafkaInternalProducer} * to commit transactions in {@link KafkaCommitter}. */ +@Internal class KafkaCommittable { private final long producerId; private final short epoch; private final String transactionalId; - @Nullable private Recyclable> producer; + @Nullable private FlinkKafkaInternalProducer producer; public KafkaCommittable( long producerId, short epoch, String transactionalId, - @Nullable Recyclable> producer) { + @Nullable FlinkKafkaInternalProducer producer) { this.producerId = producerId; this.epoch = epoch; this.transactionalId = transactionalId; this.producer = producer; } - public static KafkaCommittable of( - FlinkKafkaInternalProducer producer, - Consumer> recycler) { + public static KafkaCommittable of(FlinkKafkaInternalProducer producer) { return new KafkaCommittable( producer.getProducerId(), producer.getEpoch(), producer.getTransactionalId(), - new Recyclable<>(producer, recycler)); + producer); } public long getProducerId() { @@ -67,7 +68,7 @@ public String getTransactionalId() { return transactionalId; } - public Optional>> getProducer() { + public Optional> getProducer() { return Optional.ofNullable(producer); } @@ -78,8 +79,11 @@ public String toString() { + producerId + ", epoch=" + epoch - + ", transactionalId=" + + ", transactionalId='" + transactionalId + + '\'' + + ", producer=" + + producer + '}'; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java index 4dbeaf9e7..318e8bec6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaCommitter.java @@ -17,7 +17,13 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.InvalidTxnStateException; @@ -48,11 +54,23 @@ class KafkaCommitter implements Committer, Closeable { + "To avoid data loss, the application will restart."; private final Properties kafkaProducerConfig; + private final WritableBackchannel backchannel; + @Nullable private FlinkKafkaInternalProducer committingProducer; - @Nullable private FlinkKafkaInternalProducer recoveryProducer; - - KafkaCommitter(Properties kafkaProducerConfig) { + KafkaCommitter( + Properties kafkaProducerConfig, + String transactionalIdPrefix, + int subtaskId, + int attemptNumber) { this.kafkaProducerConfig = kafkaProducerConfig; + backchannel = + BackchannelFactory.getInstance() + .getWritableBackchannel(subtaskId, attemptNumber, transactionalIdPrefix); + } + + @VisibleForTesting + public WritableBackchannel getBackchannel() { + return backchannel; } @Override @@ -62,17 +80,12 @@ public void commit(Collection> requests) final KafkaCommittable committable = request.getCommittable(); final String transactionalId = committable.getTransactionalId(); LOG.debug("Committing Kafka transaction {}", transactionalId); - Optional>> recyclable = - committable.getProducer(); + Optional> writerProducer = committable.getProducer(); FlinkKafkaInternalProducer producer; try { - producer = - recyclable - .>map(Recyclable::getObject) - .orElseGet(() -> getRecoveryProducer(committable)); + producer = writerProducer.orElseGet(() -> getProducer(committable)); producer.commitTransaction(); - producer.flush(); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); } catch (RetriableException e) { LOG.warn( "Encountered retriable exception while committing {}.", transactionalId, e); @@ -91,7 +104,7 @@ public void commit(Collection> requests) ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, kafkaProducerConfig.getProperty(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (InvalidTxnStateException e) { // This exception only occurs when aborting after a commit or vice versa. @@ -101,30 +114,32 @@ public void commit(Collection> requests) + "Most likely the transaction has been aborted for some reason. Please check the Kafka logs for more details.", request, e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (UnknownProducerIdException e) { LOG.error( "Unable to commit transaction ({}) " + UNKNOWN_PRODUCER_ID_ERROR_MESSAGE, request, e); - recyclable.ifPresent(Recyclable::close); + backchannel.send(committable.getTransactionalId()); request.signalFailedWithKnownReason(e); } catch (Exception e) { LOG.error( "Transaction ({}) encountered error and data has been potentially lost.", request, e); - recyclable.ifPresent(Recyclable::close); + // cause failover request.signalFailedWithUnknownReason(e); } } } @Override - public void close() { - if (recoveryProducer != null) { - recoveryProducer.close(); + public void close() throws IOException { + try { + IOUtils.closeAll(backchannel, committingProducer); + } catch (Exception e) { + ExceptionUtils.rethrow(e); } } @@ -132,15 +147,15 @@ public void close() { * Creates a producer that can commit into the same transaction as the upstream producer that * was serialized into {@link KafkaCommittable}. */ - private FlinkKafkaInternalProducer getRecoveryProducer(KafkaCommittable committable) { - if (recoveryProducer == null) { - recoveryProducer = + private FlinkKafkaInternalProducer getProducer(KafkaCommittable committable) { + if (committingProducer == null) { + committingProducer = new FlinkKafkaInternalProducer<>( kafkaProducerConfig, committable.getTransactionalId()); } else { - recoveryProducer.setTransactionId(committable.getTransactionalId()); + committingProducer.setTransactionId(committable.getTransactionalId()); } - recoveryProducer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); - return recoveryProducer; + committingProducer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); + return committingProducer; } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java new file mode 100644 index 000000000..19a2d4731 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaPartitioner.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * A {@code KafkaPartitioner} wraps logic on how to partition records across partitions of multiple + * Kafka topics. + */ +@PublicEvolving +public interface KafkaPartitioner extends Serializable { + /** + * Initializer for the partitioner. This is called once on each parallel sink instance of the + * Flink Kafka producer. This method should be overridden if necessary. + * + * @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink + * @param parallelInstances the total number of parallel instances + */ + default void open(int parallelInstanceId, int parallelInstances) {} + + /** + * Determine the id of the partition that the record should be written to. + * + * @param record the record value + * @param key serialized key of the record + * @param value serialized value of the record + * @param targetTopic target topic for the record + * @param partitions found partitions for the target topic + * @return the id of the target partition + */ + int partition(T record, byte[] key, byte[] value, String targetTopic, int[] partitions); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java index 9d081c755..f56a7da54 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchema.java @@ -29,7 +29,10 @@ /** * A serialization schema which defines how to convert a value of type {@code T} to {@link - * ProducerRecord}. + * ProducerRecord}. {@link org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider} can + * be implemented to provide Kafka specific lineage metadata, while {@link + * org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider} can be implemented to provide + * lineage metadata with type information. * * @param the type of values being serialized */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java index 59864a373..0fba3a364 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilder.java @@ -19,16 +19,32 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; +import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import com.google.common.reflect.TypeToken; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.io.Serializable; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.OptionalInt; import java.util.function.Function; @@ -79,17 +95,20 @@ */ @PublicEvolving public class KafkaRecordSerializationSchemaBuilder { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); @Nullable private Function topicSelector; @Nullable private SerializationSchema valueSerializationSchema; - @Nullable private FlinkKafkaPartitioner partitioner; + @Nullable private KafkaPartitioner partitioner; @Nullable private SerializationSchema keySerializationSchema; + @Nullable private HeaderProvider headerProvider; /** * Sets a custom partitioner determining the target partition of the target topic. * * @param partitioner * @return {@code this} + * @deprecated use {@link #setPartitioner(KafkaPartitioner)} */ public KafkaRecordSerializationSchemaBuilder setPartitioner( FlinkKafkaPartitioner partitioner) { @@ -98,6 +117,19 @@ public KafkaRecordSerializationSchemaBuilder setPartitioner( return self; } + /** + * Sets a custom partitioner determining the target partition of the target topic. + * + * @param partitioner + * @return {@code this} + */ + public KafkaRecordSerializationSchemaBuilder setPartitioner( + KafkaPartitioner partitioner) { + KafkaRecordSerializationSchemaBuilder self = self(); + self.partitioner = checkNotNull(partitioner); + return self; + } + /** * Sets a fixed topic which used as destination for all records. * @@ -107,7 +139,8 @@ public KafkaRecordSerializationSchemaBuilder setPartitioner( public KafkaRecordSerializationSchemaBuilder setTopic(String topic) { checkState(this.topicSelector == null, "Topic selector already set."); checkNotNull(topic); - this.topicSelector = new CachingTopicSelector<>((e) -> topic); + + this.topicSelector = new ConstantTopicSelector<>(topic); return this; } @@ -190,6 +223,20 @@ public KafkaRecordSerializationSchemaBuilder setValueSerializa return self; } + /** + * Sets a {@link HeaderProvider} which is used to add headers to the {@link ProducerRecord} for + * the current element. + * + * @param headerProvider + * @return {@code this} + */ + public KafkaRecordSerializationSchemaBuilder setHeaderProvider( + HeaderProvider headerProvider) { + KafkaRecordSerializationSchemaBuilder self = self(); + self.headerProvider = checkNotNull(headerProvider); + return self; + } + @SuppressWarnings("unchecked") private KafkaRecordSerializationSchemaBuilder self() { return (KafkaRecordSerializationSchemaBuilder) this; @@ -239,7 +286,11 @@ public KafkaRecordSerializationSchema build() { checkState(valueSerializationSchema != null, "No value serializer is configured."); checkState(topicSelector != null, "No topic selector is configured."); return new KafkaRecordSerializationSchemaWrapper<>( - topicSelector, valueSerializationSchema, keySerializationSchema, partitioner); + topicSelector, + valueSerializationSchema, + keySerializationSchema, + partitioner, + headerProvider); } private void checkValueSerializerNotSet() { @@ -250,7 +301,29 @@ private void checkKeySerializerNotSet() { checkState(keySerializationSchema == null, "Key serializer already set."); } - private static class CachingTopicSelector implements Function, Serializable { + private static class ConstantTopicSelector + implements Function, Serializable, KafkaDatasetIdentifierProvider { + + private String topic; + + ConstantTopicSelector(String topic) { + this.topic = topic; + } + + @Override + public String apply(IN in) { + return topic; + } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics(Collections.singletonList(topic))); + } + } + + private static class CachingTopicSelector + implements Function, KafkaDatasetIdentifierProvider, Serializable { private static final int CACHE_RESET_SIZE = 5; private final Map cache; @@ -265,29 +338,43 @@ private static class CachingTopicSelector implements Function, S public String apply(IN in) { final String topic = cache.getOrDefault(in, topicSelector.apply(in)); cache.put(in, topic); - if (cache.size() == CACHE_RESET_SIZE) { + if (cache.size() >= CACHE_RESET_SIZE) { cache.clear(); } return topic; } + + @Override + public Optional getDatasetIdentifier() { + if (topicSelector instanceof KafkaDatasetIdentifierProvider) { + return ((KafkaDatasetIdentifierProvider) topicSelector).getDatasetIdentifier(); + } else { + return Optional.empty(); + } + } } private static class KafkaRecordSerializationSchemaWrapper - implements KafkaRecordSerializationSchema { + implements KafkaDatasetFacetProvider, + KafkaRecordSerializationSchema, + TypeDatasetFacetProvider { private final SerializationSchema valueSerializationSchema; private final Function topicSelector; - private final FlinkKafkaPartitioner partitioner; + private final KafkaPartitioner partitioner; private final SerializationSchema keySerializationSchema; + private final HeaderProvider headerProvider; KafkaRecordSerializationSchemaWrapper( Function topicSelector, SerializationSchema valueSerializationSchema, @Nullable SerializationSchema keySerializationSchema, - @Nullable FlinkKafkaPartitioner partitioner) { + @Nullable KafkaPartitioner partitioner, + @Nullable HeaderProvider headerProvider) { this.topicSelector = checkNotNull(topicSelector); this.valueSerializationSchema = checkNotNull(valueSerializationSchema); this.partitioner = partitioner; this.keySerializationSchema = keySerializationSchema; + this.headerProvider = headerProvider; } @Override @@ -330,7 +417,49 @@ public ProducerRecord serialize( partition.isPresent() ? partition.getAsInt() : null, timestamp == null || timestamp < 0L ? null : timestamp, key, - value); + value, + headerProvider != null ? headerProvider.getHeaders(element) : null); + } + + @Override + public Optional getKafkaDatasetFacet() { + if (!(topicSelector instanceof KafkaDatasetIdentifierProvider)) { + LOG.info("Cannot identify topics. Not an TopicsIdentifierProvider"); + return Optional.empty(); + } + + Optional topicsIdentifier = + ((KafkaDatasetIdentifierProvider) (topicSelector)).getDatasetIdentifier(); + + if (!topicsIdentifier.isPresent()) { + LOG.info("No topics' identifiers provided"); + return Optional.empty(); + } + + return Optional.of(new DefaultKafkaDatasetFacet(topicsIdentifier.get())); + } + + @Override + public Optional getTypeDatasetFacet() { + if (this.valueSerializationSchema instanceof ResultTypeQueryable) { + return Optional.of( + new DefaultTypeDatasetFacet( + ((ResultTypeQueryable) this.valueSerializationSchema) + .getProducedType())); + } else { + // gets type information from serialize method signature + TypeToken serializationSchemaType = + TypeToken.of(valueSerializationSchema.getClass()); + Class parameterType = + serializationSchemaType + .resolveType(SerializationSchema.class.getTypeParameters()[0]) + .getRawType(); + if (parameterType != Object.class) { + return Optional.of( + new DefaultTypeDatasetFacet(TypeInformation.of(parameterType))); + } + } + return Optional.empty(); } } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java index 6f74aaed5..ae498ca00 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSink.java @@ -21,14 +21,29 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Committer; -import org.apache.flink.api.connector.sink2.StatefulSink; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.LineageUtil; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.streaming.api.lineage.LineageVertexProvider; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.Optional; import java.util.Properties; /** @@ -56,9 +71,10 @@ */ @PublicEvolving public class KafkaSink - implements StatefulSink, - TwoPhaseCommittingSink { - + implements LineageVertexProvider, + TwoPhaseCommittingStatefulSink, + SupportsPostCommitTopology { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class); private final DeliveryGuarantee deliveryGuarantee; private final KafkaRecordSerializationSchema recordSerializer; @@ -86,10 +102,13 @@ public static KafkaSinkBuilder builder() { return new KafkaSinkBuilder<>(); } - @Internal @Override - public Committer createCommitter() throws IOException { - return new KafkaCommitter(kafkaProducerConfig); + public Committer createCommitter(CommitterInitContext context) { + return new KafkaCommitter( + kafkaProducerConfig, + transactionalIdPrefix, + context.getTaskInfo().getIndexOfThisSubtask(), + context.getTaskInfo().getAttemptNumber()); } @Internal @@ -101,28 +120,35 @@ public SimpleVersionedSerializer getCommittableSerializer() { @Internal @Override public KafkaWriter createWriter(InitContext context) throws IOException { - return new KafkaWriter( - deliveryGuarantee, - kafkaProducerConfig, - transactionalIdPrefix, - context, - recordSerializer, - context.asSerializationSchemaInitializationContext(), - Collections.emptyList()); + return restoreWriter(context, Collections.emptyList()); } @Internal @Override public KafkaWriter restoreWriter( - InitContext context, Collection recoveredState) throws IOException { - return new KafkaWriter<>( - deliveryGuarantee, - kafkaProducerConfig, - transactionalIdPrefix, - context, - recordSerializer, - context.asSerializationSchemaInitializationContext(), - recoveredState); + InitContext context, Collection recoveredState) { + KafkaWriter writer; + if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { + writer = + new ExactlyOnceKafkaWriter<>( + deliveryGuarantee, + kafkaProducerConfig, + transactionalIdPrefix, + context, + recordSerializer, + context.asSerializationSchemaInitializationContext(), + recoveredState); + } else { + writer = + new KafkaWriter<>( + deliveryGuarantee, + kafkaProducerConfig, + context, + recordSerializer, + context.asSerializationSchemaInitializationContext()); + } + writer.initialize(); + return writer; } @Internal @@ -131,8 +157,62 @@ public SimpleVersionedSerializer getWriterStateSerializer() { return new KafkaWriterStateSerializer(); } + @Override + public void addPostCommitTopology(DataStream> committer) { + // this is a somewhat hacky way to ensure that the committer and writer are co-located + if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE && transactionalIdPrefix != null) { + Transformation transformation = committer.getTransformation(); + // all sink transformations output CommittableMessage, so we can safely traverse the + // chain; custom colocation key is set before and should be preserved + while (transformation.getOutputType() instanceof CommittableMessageTypeInfo + && transformation.getCoLocationGroupKey() == null) { + // colocate by transactionalIdPrefix, which should be unique + transformation.setCoLocationGroupKey(transactionalIdPrefix); + transformation = transformation.getInputs().get(0); + } + } + } + @VisibleForTesting protected Properties getKafkaProducerConfig() { return kafkaProducerConfig; } + + @Override + public LineageVertex getLineageVertex() { + // enrich dataset facet with properties + Optional kafkaDatasetFacet; + if (recordSerializer instanceof KafkaDatasetFacetProvider) { + kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) recordSerializer).getKafkaDatasetFacet(); + + if (!kafkaDatasetFacet.isPresent()) { + LOG.info("Provider did not return kafka dataset facet"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + kafkaDatasetFacet.get().setProperties(this.kafkaProducerConfig); + } else { + LOG.info( + "recordSerializer does not implement KafkaDatasetFacetProvider: {}", + recordSerializer); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + String namespace = LineageUtil.namespaceOf(kafkaProducerConfig); + + Optional typeDatasetFacet = Optional.empty(); + if (recordSerializer instanceof TypeDatasetFacetProvider) { + typeDatasetFacet = ((TypeDatasetFacetProvider) recordSerializer).getTypeDatasetFacet(); + } + + if (typeDatasetFacet.isPresent()) { + return LineageUtil.sourceLineageVertexOf( + Collections.singleton( + LineageUtil.datasetOf( + namespace, kafkaDatasetFacet.get(), typeDatasetFacet.get()))); + } + + return LineageUtil.sourceLineageVertexOf( + Collections.singleton(LineageUtil.datasetOf(namespace, kafkaDatasetFacet.get()))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java index f0c20cfc0..ffd4177ba 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilder.java @@ -70,7 +70,7 @@ public class KafkaSinkBuilder { private static final int MAXIMUM_PREFIX_BYTES = 64000; private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; - private String transactionalIdPrefix = "kafka-sink"; + private String transactionalIdPrefix; private final Properties kafkaProducerConfig; private KafkaRecordSerializationSchema recordSerializer; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java index ba2cb4e48..ec8b7a223 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java @@ -22,27 +22,20 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink2.Sink; -import org.apache.flink.api.connector.sink2.StatefulSink; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.MetricUtil; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricMutableWrapper; import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava30.com.google.common.io.Closer; - import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.UnknownProducerIdException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,14 +43,11 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayDeque; import java.util.Collection; import java.util.Collections; -import java.util.Deque; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; @@ -66,14 +56,13 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * This class is responsible to write records in a Kafka topic and to handle the different delivery - * {@link DeliveryGuarantee}s. + * This class is responsible to write records in a Kafka topic without transactions. * * @param The type of the input elements. */ class KafkaWriter - implements StatefulSink.StatefulSinkWriter, - TwoPhaseCommittingSink.PrecommittingSinkWriter { + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + IN, KafkaWriterState, KafkaCommittable> { private static final Logger LOG = LoggerFactory.getLogger(KafkaWriter.class); private static final String KAFKA_PRODUCER_METRIC_NAME = "KafkaProducer"; @@ -84,12 +73,11 @@ class KafkaWriter private static final String KAFKA_PRODUCER_METRICS = "producer-metrics"; private final DeliveryGuarantee deliveryGuarantee; - private final Properties kafkaProducerConfig; - private final String transactionalIdPrefix; - private final KafkaRecordSerializationSchema recordSerializer; - private final Callback deliveryCallback; - private final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext; - + protected final Properties kafkaProducerConfig; + protected final KafkaRecordSerializationSchema recordSerializer; + protected final Callback deliveryCallback; + protected final KafkaRecordSerializationSchema.KafkaSinkContext kafkaSinkContext; + private volatile Exception asyncProducerException; private final Map previouslyCreatedMetrics = new HashMap<>(); private final SinkWriterMetricGroup metricGroup; private final boolean disabledMetrics; @@ -101,13 +89,7 @@ class KafkaWriter // Number of outgoing bytes at the latest metric sync private long latestOutgoingByteTotal; private Metric byteOutMetric; - private FlinkKafkaInternalProducer currentProducer; - private final KafkaWriterState kafkaWriterState; - // producer pool only used for exactly once - private final Deque> producerPool = - new ArrayDeque<>(); - private final Closer closer = Closer.create(); - private long lastCheckpointId; + protected FlinkKafkaInternalProducer currentProducer; private boolean closed = false; private long lastSync = System.currentTimeMillis(); @@ -121,24 +103,20 @@ class KafkaWriter * * @param deliveryGuarantee the Sink's delivery guarantee * @param kafkaProducerConfig the properties to configure the {@link FlinkKafkaInternalProducer} - * @param transactionalIdPrefix used to create the transactionalIds * @param sinkInitContext context to provide information about the runtime environment * @param recordSerializer serialize to transform the incoming records to {@link ProducerRecord} * @param schemaContext context used to initialize the {@link KafkaRecordSerializationSchema} - * @param recoveredStates state from an previous execution which was covered */ KafkaWriter( DeliveryGuarantee deliveryGuarantee, Properties kafkaProducerConfig, - String transactionalIdPrefix, Sink.InitContext sinkInitContext, KafkaRecordSerializationSchema recordSerializer, - SerializationSchema.InitializationContext schemaContext, - Collection recoveredStates) { + SerializationSchema.InitializationContext schemaContext) { this.deliveryGuarantee = checkNotNull(deliveryGuarantee, "deliveryGuarantee"); this.kafkaProducerConfig = checkNotNull(kafkaProducerConfig, "kafkaProducerConfig"); - this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix, "transactionalIdPrefix"); this.recordSerializer = checkNotNull(recordSerializer, "recordSerializer"); + checkNotNull(sinkInitContext, "sinkInitContext"); this.deliveryCallback = new WriterCallback( sinkInitContext.getMailboxExecutor(), @@ -150,7 +128,6 @@ class KafkaWriter || kafkaProducerConfig.containsKey(KEY_REGISTER_METRICS) && !Boolean.parseBoolean( kafkaProducerConfig.get(KEY_REGISTER_METRICS).toString()); - checkNotNull(sinkInitContext, "sinkInitContext"); this.timeService = sinkInitContext.getProcessingTimeService(); this.metricGroup = sinkInitContext.metricGroup(); this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); @@ -167,31 +144,17 @@ class KafkaWriter throw new FlinkRuntimeException("Cannot initialize schema.", e); } - this.kafkaWriterState = new KafkaWriterState(transactionalIdPrefix); - this.lastCheckpointId = - sinkInitContext - .getRestoredCheckpointId() - .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { - abortLingeringTransactions( - checkNotNull(recoveredStates, "recoveredStates"), lastCheckpointId + 1); - this.currentProducer = getTransactionalProducer(lastCheckpointId + 1); - this.currentProducer.beginTransaction(); - } else if (deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE - || deliveryGuarantee == DeliveryGuarantee.NONE) { - this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig, null); - closer.register(this.currentProducer); - initKafkaMetrics(this.currentProducer); - } else { - throw new UnsupportedOperationException( - "Unsupported Kafka writer semantic " + this.deliveryGuarantee); - } - initFlinkMetrics(); } + public void initialize() { + this.currentProducer = new FlinkKafkaInternalProducer<>(this.kafkaProducerConfig); + initKafkaMetrics(this.currentProducer); + } + @Override public void write(@Nullable IN element, Context context) throws IOException { + checkAsyncException(); final ProducerRecord record = recordSerializer.serialize(element, kafkaSinkContext, context.timestamp()); if (record != null) { @@ -206,57 +169,31 @@ public void flush(boolean endOfInput) throws IOException, InterruptedException { LOG.debug("final flush={}", endOfInput); currentProducer.flush(); } + + checkAsyncException(); } @Override public Collection prepareCommit() { - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { - final List committables = - Collections.singletonList( - KafkaCommittable.of(currentProducer, producerPool::add)); - LOG.debug("Committing {} committables.", committables); - return committables; - } return Collections.emptyList(); } @Override public List snapshotState(long checkpointId) throws IOException { - if (deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE) { - currentProducer = getTransactionalProducer(checkpointId + 1); - currentProducer.beginTransaction(); - } - return ImmutableList.of(kafkaWriterState); + return Collections.emptyList(); } @Override public void close() throws Exception { closed = true; LOG.debug("Closing writer with {}", currentProducer); - closeAll( - this::abortCurrentProducer, - closer, - producerPool::clear, - () -> { - checkState(currentProducer.isClosed()); - currentProducer = null; - }); - } - - private void abortCurrentProducer() { - if (currentProducer.isInTransaction()) { - try { - currentProducer.abortTransaction(); - } catch (ProducerFencedException e) { - LOG.debug( - "Producer {} fenced while aborting", currentProducer.getTransactionalId()); - } - } - } + closeAll(currentProducer); + checkState( + currentProducer.isClosed(), "Could not close current producer " + currentProducer); + currentProducer = null; - @VisibleForTesting - Deque> getProducerPool() { - return producerPool; + // Rethrow exception for the case in which close is called before writer() and flush(). + checkAsyncException(); } @VisibleForTesting @@ -264,81 +201,12 @@ FlinkKafkaInternalProducer getCurrentProducer() { return currentProducer; } - void abortLingeringTransactions( - Collection recoveredStates, long startCheckpointId) { - List prefixesToAbort = Lists.newArrayList(transactionalIdPrefix); - - final Optional lastStateOpt = recoveredStates.stream().findFirst(); - if (lastStateOpt.isPresent()) { - KafkaWriterState lastState = lastStateOpt.get(); - if (!lastState.getTransactionalIdPrefix().equals(transactionalIdPrefix)) { - prefixesToAbort.add(lastState.getTransactionalIdPrefix()); - LOG.warn( - "Transactional id prefix from previous execution {} has changed to {}.", - lastState.getTransactionalIdPrefix(), - transactionalIdPrefix); - } - } - - try (TransactionAborter transactionAborter = - new TransactionAborter( - kafkaSinkContext.getParallelInstanceId(), - kafkaSinkContext.getNumberOfParallelInstances(), - this::getOrCreateTransactionalProducer, - producerPool::add)) { - transactionAborter.abortLingeringTransactions(prefixesToAbort, startCheckpointId); - } - } - - /** - * For each checkpoint we create new {@link FlinkKafkaInternalProducer} so that new transactions - * will not clash with transactions created during previous checkpoints ({@code - * producer.initTransactions()} assures that we obtain new producerId and epoch counters). - * - *

Ensures that all transaction ids in between lastCheckpointId and checkpointId are - * initialized. - */ - private FlinkKafkaInternalProducer getTransactionalProducer(long checkpointId) { - checkState( - checkpointId > lastCheckpointId, - "Expected %s > %s", - checkpointId, - lastCheckpointId); - FlinkKafkaInternalProducer producer = null; - // in case checkpoints have been aborted, Flink would create non-consecutive transaction ids - // this loop ensures that all gaps are filled with initialized (empty) transactions - for (long id = lastCheckpointId + 1; id <= checkpointId; id++) { - String transactionalId = - TransactionalIdFactory.buildTransactionalId( - transactionalIdPrefix, kafkaSinkContext.getParallelInstanceId(), id); - producer = getOrCreateTransactionalProducer(transactionalId); - } - this.lastCheckpointId = checkpointId; - assert producer != null; - LOG.info("Created new transactional producer {}", producer.getTransactionalId()); - return producer; - } - - private FlinkKafkaInternalProducer getOrCreateTransactionalProducer( - String transactionalId) { - FlinkKafkaInternalProducer producer = producerPool.poll(); - if (producer == null) { - producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); - closer.register(producer); - producer.initTransactions(); - initKafkaMetrics(producer); - } else { - producer.initTransactionId(transactionalId); - } - return producer; - } - - private void initFlinkMetrics() { + protected void initFlinkMetrics() { metricGroup.setCurrentSendTimeGauge(this::computeSendTime); registerMetricSync(); } - private void initKafkaMetrics(FlinkKafkaInternalProducer producer) { + protected void initKafkaMetrics(FlinkKafkaInternalProducer producer) { byteOutMetric = MetricUtil.getKafkaMetric( producer.metrics(), KAFKA_PRODUCER_METRICS, "outgoing-byte-total"); @@ -397,6 +265,22 @@ private void registerMetricSync() { }); } + /** + * This method should only be invoked in the mailbox thread since the counter is not volatile. + * Logic needs to be invoked by write AND flush since we support various semantics. + */ + private void checkAsyncException() throws IOException { + // reset this exception since we could close the writer later on + Exception e = asyncProducerException; + if (e != null) { + + asyncProducerException = null; + numRecordsOutErrorsCounter.inc(); + throw new IOException( + "One or more Kafka Producer send requests have encountered exception", e); + } + } + private class WriterCallback implements Callback { private final MailboxExecutor mailboxExecutor; @Nullable private final Consumer metadataConsumer; @@ -413,12 +297,25 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { if (exception != null) { FlinkKafkaInternalProducer producer = KafkaWriter.this.currentProducer; + + // Propagate the first exception since amount of exceptions could be large. Need to + // do this in Producer IO thread since flush() guarantees that the future will + // complete. The same guarantee does not hold for tasks executed in separate + // executor e.g. mailbox executor. flush() needs to have the exception immediately + // available to fail the checkpoint. + if (asyncProducerException == null) { + asyncProducerException = decorateException(metadata, exception, producer); + } + + // Checking for exceptions from previous writes + // Notice: throwing exception in mailboxExecutor thread is not safe enough for + // triggering global fail over, which has been fixed in [FLINK-31305]. mailboxExecutor.execute( () -> { - numRecordsOutErrorsCounter.inc(); - throwException(metadata, exception, producer); + // Checking for exceptions from previous writes + checkAsyncException(); }, - "Failed to send data to Kafka"); + "Update error metric"); } if (metadataConsumer != null) { @@ -426,7 +323,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } } - private void throwException( + private FlinkRuntimeException decorateException( RecordMetadata metadata, Exception exception, FlinkKafkaInternalProducer producer) { @@ -435,7 +332,7 @@ private void throwException( if (exception instanceof UnknownProducerIdException) { message += KafkaCommitter.UNKNOWN_PRODUCER_ID_ERROR_MESSAGE; } - throw new FlinkRuntimeException(message, exception); + return new FlinkRuntimeException(message, exception); } } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java deleted file mode 100644 index 012fa992c..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/Recyclable.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.flink.connector.kafka.sink; - -import java.io.Closeable; -import java.util.function.Consumer; - -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; - -class Recyclable implements Closeable { - private T object; - private final Consumer recycler; - - public Recyclable(T object, Consumer recycler) { - this.object = checkNotNull(object); - this.recycler = checkNotNull(recycler); - } - - public T getObject() { - checkState(!isRecycled(), "Already recycled"); - return object; - } - - boolean isRecycled() { - return object == null; - } - - @Override - public void close() { - recycler.accept(object); - object = null; - } -} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java index cae6ca648..353cfea42 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionAborter.java @@ -17,6 +17,9 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; + import javax.annotation.Nullable; import java.io.Closeable; @@ -47,18 +50,18 @@ class TransactionAborter implements Closeable { private final int subtaskId; private final int parallelism; private final Function> producerFactory; - private final Consumer> closeAction; + private final Consumer> recycler; @Nullable FlinkKafkaInternalProducer producer = null; public TransactionAborter( int subtaskId, int parallelism, Function> producerFactory, - Consumer> closeAction) { + Consumer> recycler) { this.subtaskId = subtaskId; this.parallelism = parallelism; this.producerFactory = checkNotNull(producerFactory); - this.closeAction = closeAction; + this.recycler = recycler; } void abortLingeringTransactions(List prefixesToAbort, long startCheckpointId) { @@ -100,14 +103,12 @@ private int abortTransactionOfSubtask(String prefix, long startCheckpointId, int // initTransactions fences all old transactions with the same id by bumping the epoch String transactionalId = TransactionalIdFactory.buildTransactionalId(prefix, subtaskId, checkpointId); - if (producer == null) { - producer = producerFactory.apply(transactionalId); - } else { - producer.initTransactionId(transactionalId); - } + producer = producerFactory.apply(transactionalId); producer.flush(); // An epoch of 0 indicates that the id was unused before - if (producer.getEpoch() == 0) { + short epoch = producer.getEpoch(); + recycler.accept(producer); + if (epoch == 0) { // Note that the check works beyond transaction log timeouts and just depends on the // retention of the transaction topic (typically 7d). Any transaction that is not in // the that topic anymore is also not lingering (i.e., it will not block downstream @@ -123,9 +124,5 @@ private int abortTransactionOfSubtask(String prefix, long startCheckpointId, int return numTransactionAborted; } - public void close() { - if (producer != null) { - closeAction.accept(producer); - } - } + public void close() {} } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java deleted file mode 100644 index 2d79b3506..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionsToAbortChecker.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.flink.connector.kafka.sink; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -class TransactionsToAbortChecker { - - private static final int MINIMUM_CHECKPOINT_OFFSET = 1; - - private final int numberOfParallelSubtasks; - private final Map subtaskIdCheckpointOffsetMapping; - private final int subtaskId; - - TransactionsToAbortChecker( - int numberOfParallelSubtasks, - Map subtaskIdCheckpointOffsetMapping, - int subtaskId) { - this.subtaskId = subtaskId; - this.numberOfParallelSubtasks = numberOfParallelSubtasks; - this.subtaskIdCheckpointOffsetMapping = subtaskIdCheckpointOffsetMapping; - } - - /** - * Iterates through all open transactions and filters for the following attributes. - * - *

    - *
  1. If the minimum checkpointOffset for the subtask is {@link #MINIMUM_CHECKPOINT_OFFSET} - * and [openSubtaskId % {@link #numberOfParallelSubtasks} == {@link #subtaskId}] return - * all transactions from this subtask - *
  2. If the subtaskId is part of the recovered states {@link - * #subtaskIdCheckpointOffsetMapping} and the checkpointOffset >= the recovered offSet - * also return this transactionalId - *
- * - * @param openTransactions Mapping of {subtaskId: {checkpointOffset: transactionalId}} - * @return transactionalIds which must be aborted - */ - public List getTransactionsToAbort(Map> openTransactions) { - final List transactionalIdsToAbort = new ArrayList<>(); - for (final Map.Entry> subtaskOffsetMapping : - openTransactions.entrySet()) { - final Map checkpointOffsetTransactionalIdMapping = - subtaskOffsetMapping.getValue(); - // All transactions from this subtask have been closed - if (checkpointOffsetTransactionalIdMapping.isEmpty()) { - continue; - } - // Abort all open transactions if checkpointOffset 0 is open implying that no checkpoint - // finished. - // Cut the transactions in ranges to speed up abort process - if (Collections.min(checkpointOffsetTransactionalIdMapping.keySet()) - == MINIMUM_CHECKPOINT_OFFSET - && subtaskOffsetMapping.getKey() % numberOfParallelSubtasks == subtaskId) { - transactionalIdsToAbort.addAll(checkpointOffsetTransactionalIdMapping.values()); - } else { - // Check all open transactions against recovered ones and close if the open - // transaction is equal or higher to the offset - for (final Map.Entry offsetTransactionId : - checkpointOffsetTransactionalIdMapping.entrySet()) { - if (!hasSameSubtaskWithHigherCheckpoint( - subtaskOffsetMapping.getKey(), offsetTransactionId.getKey())) { - continue; - } - transactionalIdsToAbort.add(offsetTransactionId.getValue()); - } - } - } - return transactionalIdsToAbort; - } - - private boolean hasSameSubtaskWithHigherCheckpoint( - int openSubtaskIndex, long openCheckpointOffset) { - return subtaskIdCheckpointOffsetMapping.containsKey(openSubtaskIndex) - && subtaskIdCheckpointOffsetMapping.get(openSubtaskIndex) <= openCheckpointOffset; - } -} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java new file mode 100644 index 000000000..4e64c8698 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TwoPhaseCommittingStatefulSink.java @@ -0,0 +1,34 @@ +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; + +import java.io.IOException; +import java.util.Collection; + +/** + * A combination of {@link TwoPhaseCommittingSink} and {@link StatefulSink}. + * + *

The purpose of this interface is to be able to pass an interface rather than a {@link + * KafkaSink} implementation into the reducing sink which simplifies unit testing. + * + * @param The type of the sink's input + * @param The type of the sink writer's state + * @param The type of the committables. + */ +@Internal +public interface TwoPhaseCommittingStatefulSink + extends TwoPhaseCommittingSink, StatefulSink { + + PrecommittingStatefulSinkWriter createWriter(InitContext context) + throws IOException; + + PrecommittingStatefulSinkWriter restoreWriter( + InitContext context, Collection recoveredState) throws IOException; + + /** A combination of {@link PrecommittingSinkWriter} and {@link StatefulSinkWriter}. */ + interface PrecommittingStatefulSinkWriter + extends PrecommittingSinkWriter, + StatefulSinkWriter {} +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java new file mode 100644 index 000000000..244b76683 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/Backchannel.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import java.io.Closeable; + +/** + * A backchannel for communication between the commiter -> writer. It's used to signal that certain + * transactions have been committed and respective producers are good to be reused. + * + *

The model closely follows the idea of statefun except that there is no need to checkpoint the + * state since the backchannel will fully recover on restart from the committer state. + * + *

Establishing a backchannel for Kafka sink works because there is only writer and committer and + * nothing in between these two operators. In most cases, these two are chained in live inside the + * same task thread. In rare cases, committer and writer are not chained, so writer and committer + * are in different tasks and threads. However, because of colocations of tasks, we still know that + * both instances will run inside the same JVM and we can establish a backchannel between them. The + * latter case requires some synchronization in the buffer. + */ +public interface Backchannel extends Closeable { + /** Check if the backchannel is fully established. */ + boolean isEstablished(); + + @Override + void close(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java new file mode 100644 index 000000000..65c2bdf80 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelFactory.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple3; + +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; + +/** + * Creates and manages backchannels for the Kafka sink. The backchannels are used to communicate + * between the Kafka committer and writer. + * + *

Each backchannel is uniquely identified by the subtask id, attempt number, and transactional + * id prefix. The subtask id prevents concurrent subtasks of the same sink from conflicting. The + * attempt number prevents conflicts for task-local recovery. The transactional id prefix prevents + * conflicts between multiple Kafka sinks. + */ +@Internal +@ThreadSafe +public class BackchannelFactory { + private static final BackchannelFactory INSTANCE = new BackchannelFactory(); + + /** Gets the singleton instance of the {@link BackchannelFactory}. */ + public static BackchannelFactory getInstance() { + return INSTANCE; + } + + private BackchannelFactory() {} + + /** + * The map of backchannels, keyed by the subtask id and transactional id prefix to uniquely + * identify the backchannel while establishing the connection. + */ + private final Map, BackchannelImpl> backchannels = + new ConcurrentHashMap<>(); + + /** + * Gets a {@link ReadableBackchannel} for the given subtask, attempt, and transactional id + * prefix. + * + *

If this method is called twice with the same arguments, it will throw an exception as it + * indicates that the transactional id prefix is being reused for multiple Kafka sinks. + * + *

If the corresponding {@link #getWritableBackchannel(int, int, String)} is called, the + * {@link ReadableBackchannel#isEstablished()} will return true. + */ + @SuppressWarnings("unchecked") + public ReadableBackchannel getReadableBackchannel( + int subtaskId, int attemptNumber, String transactionalIdPrefix) { + return (ReadableBackchannel) + getBackchannel( + subtaskId, + attemptNumber, + transactionalIdPrefix, + BackchannelImpl::createReadableBackchannel); + } + + /** + * Gets a {@link WritableBackchannel} for the given subtask, attempt, and transactional id + * prefix. + * + *

If this method is called twice with the same arguments, it will throw an exception as it + * indicates that the transactional id prefix is being reused for multiple Kafka sinks. + * + *

If the corresponding {@link #getReadableBackchannel(int, int, String)} is called, the + * {@link WritableBackchannel#isEstablished()} will return true. + */ + @SuppressWarnings("unchecked") + public WritableBackchannel getWritableBackchannel( + int subtaskId, int attemptNumber, String transactionalIdPrefix) { + return (WritableBackchannel) + getBackchannel( + subtaskId, + attemptNumber, + transactionalIdPrefix, + BackchannelImpl::createWritableBackchannel); + } + + private R getBackchannel( + int subtaskId, + int attemptNumber, + String transactionalIdPrefix, + Function, R> subchannelCreator) { + Tuple3 id = + new Tuple3<>(subtaskId, attemptNumber, transactionalIdPrefix); + BackchannelImpl backchannel = + backchannels.computeIfAbsent(id, k -> new BackchannelImpl<>(() -> unregister(id))); + try { + return subchannelCreator.apply(backchannel); + } catch (Exception e) { + throw new IllegalStateException( + String.format( + "Found duplicate transactionalIdPrefix for multiple Kafka sinks: %s. Transactional id prefixes need to be unique. You may experience memory leaks without fixing this.", + transactionalIdPrefix), + e); + } + } + + private void unregister(Tuple3 id) { + backchannels.remove(id); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java new file mode 100644 index 000000000..a6996000c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImpl.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Deque; +import java.util.concurrent.ConcurrentLinkedDeque; + +/** + * A backchannel for communication between the Kafka committer -> writer. It's used to recycle + * producer and signal that certain transactions have been committed on recovery. + * + *

A backchannel provides two views: a readable view for the committer to read messages, and a + * writable view for the writer to send messages. Both views have a separate lifecycle. The + * lifecycle of this backchannel is bound to the joint lifecycle of the readable and writable views. + */ +@Internal +@ThreadSafe +public final class BackchannelImpl { + /** + * The action to be executed when the backchannel is closed. The channel is classed if both + * readable and writable channels are closed. + */ + private final Runnable closeAction; + /** + * The messages to be sent from the writer to the committer. It's a thread safe deque in case + * committer and writer are not chained. + */ + private final Deque messages = new ConcurrentLinkedDeque<>(); + + /** The readable backchannel. */ + private volatile ReadableBackchannel readableBackchannel; + /** The writable backchannel. */ + private volatile WritableBackchannel writableBackchannel; + + BackchannelImpl(Runnable closeAction) { + this.closeAction = closeAction; + } + + /** + * True iff the backchannel is established, i.e. both readable and writable channels are + * created. + */ + private boolean isEstablished() { + return readableBackchannel != null && writableBackchannel != null; + } + + /** + * Closes the readable channel. If the writable channel is also closed, the backchannel is + * closed. + */ + private void closeReadableChannel() { + if (readableBackchannel == null) { + throw new IllegalStateException("Readable backchannel does not exist."); + } + readableBackchannel = null; + checkClosed(); + } + + /** Checks if the backchannel is considered closed. If so, executes the close action. */ + private void checkClosed() { + if (readableBackchannel == null && writableBackchannel == null) { + closeAction.run(); + } + } + + ReadableBackchannel createReadableBackchannel() { + if (readableBackchannel != null) { + throw new IllegalStateException("Readable backchannel already exists."); + } + return readableBackchannel = new Readable(); + } + + WritableBackchannel createWritableBackchannel() { + if (writableBackchannel != null) { + throw new IllegalStateException("Writable backchannel already exists."); + } + return writableBackchannel = new Writable(); + } + + private void closeWritableChannel() { + if (writableBackchannel == null) { + throw new IllegalStateException("Writable backchannel does not exist."); + } + writableBackchannel = null; + checkClosed(); + } + + private class Writable implements WritableBackchannel { + @Override + public void send(T message) { + messages.add(message); + } + + public boolean isEstablished() { + return BackchannelImpl.this.isEstablished(); + } + + @Override + public void close() { + closeWritableChannel(); + } + } + + private class Readable implements ReadableBackchannel { + @Nullable + @Override + public T poll() { + return messages.poll(); + } + + @Override + public boolean isEstablished() { + return BackchannelImpl.this.isEstablished(); + } + + @Override + public void close() { + closeReadableChannel(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java new file mode 100644 index 000000000..f4fc7080d --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/CheckpointTransaction.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An immutable class that represents a transactional id and a checkpoint id. It's used inside the + * {@link ProducerPoolImpl} to keep track of the transactions that are currently in-flight. The + * checkpoint id is used to subsume committed transactions wrt to recycling producers. + */ +@Internal +public class CheckpointTransaction { + private final String transactionalId; + private final long checkpointId; + + public CheckpointTransaction(String transactionalId, long checkpointId) { + this.transactionalId = checkNotNull(transactionalId, "transactionalId must not be null"); + this.checkpointId = checkpointId; + } + + public long getCheckpointId() { + return checkpointId; + } + + public String getTransactionalId() { + return transactionalId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CheckpointTransaction that = (CheckpointTransaction) o; + return checkpointId == that.checkpointId + && Objects.equals(transactionalId, that.transactionalId); + } + + @Override + public int hashCode() { + return Objects.hash(transactionalId, checkpointId); + } + + @Override + public String toString() { + return "CheckpointTransaction{" + + "transactionalId='" + + transactionalId + + '\'' + + ", checkpointId=" + + checkpointId + + '}'; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java similarity index 73% rename from flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java rename to flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java index 7a3ed5698..a6a443049 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/FlinkKafkaInternalProducer.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + * 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, @@ -15,10 +16,13 @@ * limitations under the License. */ -package org.apache.flink.connector.kafka.sink; +package org.apache.flink.connector.kafka.sink.internal; +import org.apache.kafka.clients.producer.Callback; 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.clients.producer.internals.TransactionManager; import org.apache.kafka.clients.producer.internals.TransactionalRequestResult; import org.apache.kafka.common.errors.ProducerFencedException; @@ -33,14 +37,14 @@ import java.lang.reflect.Method; import java.time.Duration; import java.util.Properties; +import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkState; /** * A {@link KafkaProducer} that exposes private fields to allow resume producing from a given state. */ -class FlinkKafkaInternalProducer extends KafkaProducer { - +public class FlinkKafkaInternalProducer extends KafkaProducer { private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class); private static final String TRANSACTION_MANAGER_FIELD_NAME = "transactionManager"; private static final String TRANSACTION_MANAGER_STATE_ENUM = @@ -49,24 +53,35 @@ class FlinkKafkaInternalProducer extends KafkaProducer { @Nullable private String transactionalId; private volatile boolean inTransaction; + private volatile boolean hasRecordsInTransaction; private volatile boolean closed; - public FlinkKafkaInternalProducer(Properties properties, @Nullable String transactionalId) { + public FlinkKafkaInternalProducer(Properties properties) { + super(properties); + LOG.debug("Created non-transactional {}", this); + } + + public FlinkKafkaInternalProducer(Properties properties, String transactionalId) { super(withTransactionalId(properties, transactionalId)); this.transactionalId = transactionalId; + LOG.debug("Created transactional {}", this); } - private static Properties withTransactionalId( - Properties properties, @Nullable String transactionalId) { - if (transactionalId == null) { - return properties; - } + private static Properties withTransactionalId(Properties properties, String transactionalId) { Properties props = new Properties(); props.putAll(properties); props.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); return props; } + @Override + public Future send(ProducerRecord record, Callback callback) { + if (inTransaction) { + hasRecordsInTransaction = true; + } + return super.send(record, callback); + } + @Override public void flush() { super.flush(); @@ -78,6 +93,7 @@ public void flush() { @Override public void beginTransaction() throws ProducerFencedException { super.beginTransaction(); + LOG.debug("beginTransaction {}", transactionalId); inTransaction = true; } @@ -86,6 +102,7 @@ public void abortTransaction() throws ProducerFencedException { LOG.debug("abortTransaction {}", transactionalId); checkState(inTransaction, "Transaction was not started"); inTransaction = false; + hasRecordsInTransaction = false; super.abortTransaction(); } @@ -94,6 +111,7 @@ public void commitTransaction() throws ProducerFencedException { LOG.debug("commitTransaction {}", transactionalId); checkState(inTransaction, "Transaction was not started"); inTransaction = false; + hasRecordsInTransaction = false; super.commitTransaction(); } @@ -101,25 +119,26 @@ public boolean isInTransaction() { return inTransaction; } + public boolean hasRecordsInTransaction() { + return hasRecordsInTransaction; + } + @Override public void close() { - closed = true; - if (inTransaction) { - // This is state is most likely reached in case of a failure. - // If this producer is still in transaction, it should be committing. - // However, at this point, we cannot decide that and we shouldn't prolong cancellation. - // So hard kill this producer with all resources. + if (!closed) { + LOG.debug("Closing immediately {}", this); super.close(Duration.ZERO); - } else { - // If this is outside of a transaction, we should be able to cleanly shutdown. - super.close(Duration.ofHours(1)); + closed = true; } } @Override public void close(Duration timeout) { - closed = true; - super.close(timeout); + if (!closed) { + LOG.debug("Closing with {} timeout {}", timeout, this); + super.close(timeout); + closed = true; + } } public boolean isClosed() { @@ -143,35 +162,51 @@ public long getProducerId() { return (long) getField(producerIdAndEpoch, "producerId"); } - public void initTransactionId(String transactionalId) { - if (!transactionalId.equals(this.transactionalId)) { - setTransactionId(transactionalId); - initTransactions(); + /** + * Sets the transaction manager state to uninitialized. + * + *

Can only be called if the producer is in a transaction. Its main purpose is to resolve the + * split brain scenario between writer and committer. + */ + public void resetTransactionState() { + checkState(inTransaction, "Not in transactional state"); + this.inTransaction = false; + this.hasRecordsInTransaction = false; + Object transactionManager = getTransactionManager(); + synchronized (transactionManager) { + setField(transactionManager, "transactionalId", transactionalId); + setField( + transactionManager, + "currentState", + getTransactionManagerState("UNINITIALIZED")); } } + /** + * Sets the transactional id and sets the transaction manager state to uninitialized. + * + *

Can only be called if the producer is not in a transaction. + */ public void setTransactionId(String transactionalId) { - if (!transactionalId.equals(this.transactionalId)) { - checkState( - !inTransaction, - String.format("Another transaction %s is still open.", transactionalId)); - LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId); - Object transactionManager = getTransactionManager(); - synchronized (transactionManager) { - setField(transactionManager, "transactionalId", transactionalId); - setField( - transactionManager, - "currentState", - getTransactionManagerState("UNINITIALIZED")); - this.transactionalId = transactionalId; - } + checkState( + !inTransaction, + String.format("Another transaction %s is still open.", transactionalId)); + LOG.debug("Change transaction id from {} to {}", this.transactionalId, transactionalId); + this.transactionalId = transactionalId; + Object transactionManager = getTransactionManager(); + synchronized (transactionManager) { + setField(transactionManager, "transactionalId", transactionalId); + setField( + transactionManager, + "currentState", + getTransactionManagerState("UNINITIALIZED")); } } /** * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction} * is also adding new partitions to the transaction. flushNewPartitions method is moving this - * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise resumeTransaction + * logic to pre-commit/flush, to make resumeTransaction simpler. Otherwise, resumeTransaction * would require to restore state of the not yet added/"in-flight" partitions. */ private void flushNewPartitions() { @@ -301,8 +336,18 @@ public void resumeTransaction(long producerId, short epoch) { transitionTransactionManagerStateTo(transactionManager, "READY"); transitionTransactionManagerStateTo(transactionManager, "IN_TRANSACTION"); + + // the transactionStarted flag in the KafkaProducer controls whether + // an EndTxnRequest will actually be sent to Kafka for a commit + // or abort API call. This flag is set only after the first send (i.e. + // only if data is actually written to some partition). + // In checkpoints, we only ever store metadata of pre-committed + // transactions that actually have records; therefore, on restore + // when we create recovery producers to resume transactions and commit + // them, we should always set this flag. setField(transactionManager, "transactionStarted", true); this.inTransaction = true; + this.hasRecordsInTransaction = true; } } @@ -362,13 +407,8 @@ private static void transitionTransactionManagerStateTo( @Override public String toString() { - return "FlinkKafkaInternalProducer{" - + "transactionalId='" - + transactionalId - + "', inTransaction=" - + inTransaction - + ", closed=" - + closed - + '}'; + return String.format( + "FlinkKafkaInternalProducer@%d{transactionalId='%s', inTransaction=%s, closed=%s}", + System.identityHashCode(this), transactionalId, inTransaction, closed); } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java new file mode 100644 index 000000000..bece0ddc7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPool.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +/** A pool of producers that can be recycled. */ +public interface ProducerPool extends AutoCloseable { + /** + * Notify the pool that a transaction has finished. The producer with the given transactional id + * can be recycled. + */ + void recycleByTransactionId(String transactionalId); + + /** + * Get a producer for the given transactional id and checkpoint id. The producer is not recycled + * until it is passed to the committer, the committer commits the transaction, and {@link + * #recycleByTransactionId(String)} is called. Alternatively, the producer can be recycled by + * {@link #recycle(FlinkKafkaInternalProducer)}. + */ + FlinkKafkaInternalProducer getTransactionalProducer( + String transactionalId, long checkpointId); + + /** + * Explicitly recycle a producer. This is useful when the producer has not been passed to the + * committer. + */ + void recycle(FlinkKafkaInternalProducer producer); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java new file mode 100644 index 000000000..a3a2eb925 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImpl.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.annotation.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Comparator; +import java.util.Deque; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Properties; +import java.util.TreeMap; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.apache.flink.util.IOUtils.closeAll; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Manages a pool of {@link FlinkKafkaInternalProducer} instances for reuse in the {@code + * KafkaWriter} and keeps track of the used transactional ids. + * + *

Reusing the producers is important for performance reasons. The producer initialization + * includes a few requests to the broker (e.g., ApiVersion), which can be avoided with reuse. + * + *

Tracking the transactional ids in use can be a tricky because the {@code KafkaCommitter} is + * ultimately finishing the transactions. There are two major cases: + * + *

    + *
  • The committer is chained to the writer (common case): The {@code KafkaCommittable} contains + * the producer (in-memory transfer) and the producer is only returned to the producer pool + * upon completion by the committer. Thus, none of the producers in the pool have active + * transactions. + *
  • The committer is not chained: The {@code KafkaCommittableSerializer} will return the + * producer to this pool, but it still has an ongoing transaction. The producer will be + * "cloned" in the committer by using producer id and epoch. In this case, we rely on {@link + * org.apache.kafka.common.errors.ProducerFencedException} to test later if a producer in the + * pool is still in the transaction or not. + *
+ * + *

This pool is not thread-safe and is only intended to be accessed from the writer, which owns + * it. + */ +@NotThreadSafe +public class ProducerPoolImpl implements ProducerPool { + private static final Logger LOG = LoggerFactory.getLogger(ProducerPoolImpl.class); + + /** + * The configuration for the Kafka producer. This is used to create new producers when the pool + * is empty. + */ + private final Properties kafkaProducerConfig; + /** Callback to allow the writer to init metrics. */ + private final Consumer> producerInit; + /** + * The pool of producers that are available for reuse. This pool is used to avoid creating new + * producers for every transaction. + */ + private final Deque> producerPool = + new ArrayDeque<>(); + /** + * The map of ongoing transactions (id -> producer/CheckpointTransaction). This is used to keep + * track of the transactions that are ongoing and the respective producers are not in the pool. + */ + private final Map producerByTransactionalId = new TreeMap<>(); + /** + * A secondary tracking structure to quickly find transactions coming from an earlier + * checkpoints. + */ + private final NavigableMap transactionalIdsByCheckpoint = + new TreeMap<>(Comparator.comparing(CheckpointTransaction::getCheckpointId)); + + /** Creates a new {@link ProducerPoolImpl}. */ + public ProducerPoolImpl( + Properties kafkaProducerConfig, + Consumer> producerInit) { + this.kafkaProducerConfig = + checkNotNull(kafkaProducerConfig, "kafkaProducerConfig must not be null"); + this.producerInit = checkNotNull(producerInit, "producerInit must not be null"); + } + + @Override + public void recycleByTransactionId(String transactionalId) { + ProducerEntry producerEntry = producerByTransactionalId.remove(transactionalId); + LOG.debug("Transaction {} finished, producer {}", transactionalId, producerEntry); + if (producerEntry == null) { + // during recovery, the committer may finish transactions that are not yet ongoing from + // the writer's perspective + return; + } + + transactionalIdsByCheckpoint.remove(producerEntry.getCheckpointedTransaction()); + recycleProducer(producerEntry.getProducer()); + + // In rare cases (only for non-chained committer), some transactions may not be detected to + // be finished. + // For example, a transaction may be committed at the same time the writer state is + // snapshot. The writer contains the transaction as ongoing but the committer state will + // later not contain it. + // In these cases, we make use of the fact that committables are processed in order of the + // checkpoint id. + // That means a transaction state with checkpoint id C implies that all C' < C are finished. + NavigableMap earlierTransactions = + transactionalIdsByCheckpoint.headMap( + producerEntry.getCheckpointedTransaction(), false); + if (!earlierTransactions.isEmpty()) { + for (String id : earlierTransactions.values()) { + ProducerEntry entry = producerByTransactionalId.remove(id); + recycleProducer(entry.getProducer()); + } + earlierTransactions.clear(); + } + } + + @Override + public void recycle(FlinkKafkaInternalProducer producer) { + recycleProducer(producer); + ProducerEntry producerEntry = + producerByTransactionalId.remove(producer.getTransactionalId()); + transactionalIdsByCheckpoint.remove(producerEntry.getCheckpointedTransaction()); + } + + private void recycleProducer(@Nullable FlinkKafkaInternalProducer producer) { + // In case of recovery, we don't create a producer for the ongoing transactions. + // The producer is just initialized on committer side. + if (producer == null) { + return; + } + // For non-chained committer, we have a split brain scenario: + // Both the writer and the committer have a producer representing the same transaction. + // The committer producer has finished the transaction while the writer producer is still in + // transaction. In this case, we forcibly complete the transaction, such that we can + // initialize it. + if (producer.isInTransaction()) { + producer.resetTransactionState(); + } + producerPool.add(producer); + + LOG.debug("Recycling {}, new pool size {}", producer, producerPool.size()); + } + + @Override + public FlinkKafkaInternalProducer getTransactionalProducer( + String transactionalId, long checkpointId) { + FlinkKafkaInternalProducer producer = producerPool.poll(); + if (producer == null) { + producer = new FlinkKafkaInternalProducer<>(kafkaProducerConfig, transactionalId); + producerInit.accept(producer); + } else if (transactionalId != null) { + producer.setTransactionId(transactionalId); + } + if (transactionalId != null) { + // first keep track of the transaction and producer because initTransaction may be + // interrupted + CheckpointTransaction checkpointedTransaction = + new CheckpointTransaction(transactionalId, checkpointId); + ProducerEntry existing = + producerByTransactionalId.put( + transactionalId, new ProducerEntry(producer, checkpointedTransaction)); + transactionalIdsByCheckpoint.put(checkpointedTransaction, transactionalId); + checkState( + existing == null, + "Transaction %s already ongoing existing producer %s; new producer %s", + transactionalId, + existing, + producer); + producer.initTransactions(); + } + LOG.debug("getProducer {}, new pool size {}", producer, producerPool.size()); + return producer; + } + + @VisibleForTesting + public Collection> getProducers() { + return producerPool; + } + + @Override + public void close() throws Exception { + LOG.debug( + "Closing used producers {} and free producers {}", + producerByTransactionalId, + producerPool); + closeAll( + () -> closeAll(producerPool), + () -> + closeAll( + producerByTransactionalId.values().stream() + .map(ProducerEntry::getProducer) + .collect(Collectors.toList())), + producerPool::clear, + producerByTransactionalId::clear); + } + + private static class ProducerEntry { + private final FlinkKafkaInternalProducer producer; + private final CheckpointTransaction checkpointedTransaction; + + private ProducerEntry( + FlinkKafkaInternalProducer producer, + CheckpointTransaction checkpointedTransaction) { + this.producer = checkNotNull(producer, "producer must not be null"); + this.checkpointedTransaction = + checkNotNull( + checkpointedTransaction, "checkpointedTransaction must not be null"); + } + + public CheckpointTransaction getCheckpointedTransaction() { + return checkpointedTransaction; + } + + public FlinkKafkaInternalProducer getProducer() { + return producer; + } + + @Override + public String toString() { + return producer.toString(); + } + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java new file mode 100644 index 000000000..0f2cbf17c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/ReadableBackchannel.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import javax.annotation.Nullable; + +/** + * The readable portion of a backchannel for communication between the commiter -> writer. It's used + * to signal that certain transactions have been committed and respective producers are good to be + * reused. + */ +public interface ReadableBackchannel extends Backchannel { + /** + * Poll the next message from the backchannel. This method is non-blocking and returns {@code + * null} if there is no message available. + */ + @Nullable + T poll(); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java similarity index 62% rename from flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java rename to flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java index eda4c0180..1c0c6263f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/TransactionalIdFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/TransactionalIdFactory.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + * 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, @@ -15,9 +16,13 @@ * limitations under the License. */ -package org.apache.flink.connector.kafka.sink; +package org.apache.flink.connector.kafka.sink.internal; -class TransactionalIdFactory { +import org.apache.flink.annotation.Internal; + +/** Utility class for constructing transactionalIds for Kafka transactions. */ +@Internal +public class TransactionalIdFactory { private static final String TRANSACTIONAL_ID_DELIMITER = "-"; /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java new file mode 100644 index 000000000..97a0b2c8a --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/internal/WritableBackchannel.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +/** + * The writable portion of a {@link Backchannel} for communication between the commiter -> writer. + * It's used to signal that certain transactions have been committed and respective producers are + * good to be reused. + * + *

Messages can be sent before the backchannel is established. They will be consumed once the + * backchannel is established. + */ +public interface WritableBackchannel extends Backchannel { + /** Send a message to the backchannel. */ + void send(T message); +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java new file mode 100644 index 000000000..0e29576cc --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaPropertiesUtil.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.source; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nonnull; + +import java.util.Properties; + +/** Utility class for modify Kafka properties. */ +@Internal +public class KafkaPropertiesUtil { + + private KafkaPropertiesUtil() {} + + public static void copyProperties(@Nonnull Properties from, @Nonnull Properties to) { + for (String key : from.stringPropertyNames()) { + to.setProperty(key, from.getProperty(key)); + } + } + + /** + * client.id is used for Kafka server side logging, see + * https://docs.confluent.io/platform/current/installation/configuration/consumer-configs.html#consumerconfigs_client.id + * + *

Set client id prefix to avoid mbean collision warning logs. There are multiple instances + * of the AdminClient/KafkaConsumer so each instance requires a different client id (i.e. also + * per cluster). + * + *

Flink internally configures the clientId, making this the only way to customize the Kafka + * client id parameter. + * + *

If this is not done, we will encounter warning logs of the form: + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.consumer:type=app-info,id=null-enumerator-consumer + * + *

WARN org.apache.kafka.common.utils.AppInfoParser [] - Error registering AppInfo mbean + * javax.management.InstanceAlreadyExistsException: + * kafka.admin.client:type=app-info,id=null-enumerator-admin-client + */ + public static void setClientIdPrefix(Properties properties, String kafkaClusterId) { + String userClientIdPrefix = + properties.getProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key()); + properties.setProperty( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + userClientIdPrefix + "-" + kafkaClusterId); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java index 7a17b1ff6..39302751c 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSource.java @@ -33,6 +33,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.LineageUtil; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; @@ -48,14 +53,21 @@ import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.lineage.LineageVertexProvider; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; import org.apache.flink.util.UserCodeClassLoader; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.io.IOException; import java.util.Collection; +import java.util.Collections; +import java.util.Optional; import java.util.Properties; import java.util.function.Consumer; import java.util.function.Supplier; @@ -85,8 +97,10 @@ */ @PublicEvolving public class KafkaSource - implements Source, + implements LineageVertexProvider, + Source, ResultTypeQueryable { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); private static final long serialVersionUID = -8755372893283732098L; // Users can choose only one of the following ways to specify the topics to consume from. private final KafkaSubscriber subscriber; @@ -98,6 +112,8 @@ public class KafkaSource private final KafkaRecordDeserializationSchema deserializationSchema; // The configurations. private final Properties props; + // Client rackId callback + private final SerializableSupplier rackIdSupplier; KafkaSource( KafkaSubscriber subscriber, @@ -105,13 +121,15 @@ public class KafkaSource @Nullable OffsetsInitializer stoppingOffsetsInitializer, Boundedness boundedness, KafkaRecordDeserializationSchema deserializationSchema, - Properties props) { + Properties props, + SerializableSupplier rackIdSupplier) { this.subscriber = subscriber; this.startingOffsetsInitializer = startingOffsetsInitializer; this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; this.boundedness = boundedness; this.deserializationSchema = deserializationSchema; this.props = props; + this.rackIdSupplier = rackIdSupplier; } /** @@ -157,7 +175,14 @@ public UserCodeClassLoader getUserCodeClassLoader() { new KafkaSourceReaderMetrics(readerContext.metricGroup()); Supplier splitReaderSupplier = - () -> new KafkaPartitionSplitReader(props, readerContext, kafkaSourceReaderMetrics); + () -> + new KafkaPartitionSplitReader( + props, + readerContext, + kafkaSourceReaderMetrics, + Optional.ofNullable(rackIdSupplier) + .map(Supplier::get) + .orElse(null)); KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); return new KafkaSourceReader<>( @@ -196,7 +221,7 @@ public SplitEnumerator restoreEnumera props, enumContext, boundedness, - checkpoint.assignedPartitions()); + checkpoint); } @Internal @@ -238,4 +263,31 @@ KafkaSubscriber getKafkaSubscriber() { OffsetsInitializer getStoppingOffsetsInitializer() { return stoppingOffsetsInitializer; } + + @Override + public SourceLineageVertex getLineageVertex() { + if (!(subscriber instanceof KafkaDatasetIdentifierProvider)) { + LOG.info("unable to determine topic identifier"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + Optional topicsIdentifier = + ((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier(); + + if (!topicsIdentifier.isPresent()) { + LOG.info("No topics' identifier returned from subscriber"); + return LineageUtil.sourceLineageVertexOf(Collections.emptyList()); + } + + DefaultKafkaDatasetFacet kafkaDatasetFacet = + new DefaultKafkaDatasetFacet(topicsIdentifier.get(), props); + + String namespace = LineageUtil.namespaceOf(props); + return LineageUtil.sourceLineageVertexOf( + Collections.singletonList( + LineageUtil.datasetOf( + namespace, + kafkaDatasetFacet, + new DefaultTypeDatasetFacet(getProducedType())))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java index afaa72db7..0709afe0b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilder.java @@ -26,13 +26,17 @@ import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator; import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.Deserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -80,6 +84,7 @@ * .setTopics(Arrays.asList(TOPIC1, TOPIC2)) * .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)) * .setUnbounded(OffsetsInitializer.latest()) + * .setRackId(() -> MY_RACK_ID) * .build(); * } * @@ -100,6 +105,8 @@ public class KafkaSourceBuilder { private KafkaRecordDeserializationSchema deserializationSchema; // The configurations. protected Properties props; + // Client rackId supplier + private SerializableSupplier rackIdSupplier; KafkaSourceBuilder() { this.subscriber = null; @@ -108,6 +115,7 @@ public class KafkaSourceBuilder { this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; this.deserializationSchema = null; this.props = new Properties(); + this.rackIdSupplier = null; } /** @@ -355,6 +363,17 @@ public KafkaSourceBuilder setClientIdPrefix(String prefix) { return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); } + /** + * Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader. + * + * @param rackIdCallback callback to provide Kafka consumer client.rack + * @return this KafkaSourceBuilder + */ + public KafkaSourceBuilder setRackIdSupplier(SerializableSupplier rackIdCallback) { + this.rackIdSupplier = rackIdCallback; + return this; + } + /** * Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. @@ -363,8 +382,6 @@ public KafkaSourceBuilder setClientIdPrefix(String prefix) { * created. * *

    - *
  • key.deserializer is always set to {@link ByteArrayDeserializer}. - *
  • value.deserializer is always set to {@link ByteArrayDeserializer}. *
  • auto.offset.reset.strategy is overridden by {@link * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by * default {@link OffsetsInitializer#earliest()}. @@ -389,8 +406,6 @@ public KafkaSourceBuilder setProperty(String key, String value) { * created. * *
      - *
    • key.deserializer is always set to {@link ByteArrayDeserializer}. - *
    • value.deserializer is always set to {@link ByteArrayDeserializer}. *
    • auto.offset.reset.strategy is overridden by {@link * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by * default {@link OffsetsInitializer#earliest()}. @@ -422,7 +437,8 @@ public KafkaSource build() { stoppingOffsetsInitializer, boundedness, deserializationSchema, - props); + props, + rackIdSupplier); } // ------------- private helpers -------------- @@ -440,11 +456,11 @@ private void parseAndSetRequiredProperties() { maybeOverride( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - true); + false); maybeOverride( ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - true); + false); if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { LOG.warn( "Offset commit on checkpoint is disabled because {} is not specified", @@ -458,10 +474,9 @@ private void parseAndSetRequiredProperties() { true); // If the source is bounded, do not run periodic partition discovery. - maybeOverride( - KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - "-1", - boundedness == Boundedness.BOUNDED); + if (boundedness == Boundedness.BOUNDED) { + maybeOverride(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "-1", true); + } // If the client id prefix is not set, reuse the consumer group id as the client id prefix, // or generate a random string if consumer group id is not specified. @@ -517,6 +532,47 @@ private void sanityCheck() { if (stoppingOffsetsInitializer instanceof OffsetsInitializerValidator) { ((OffsetsInitializerValidator) stoppingOffsetsInitializer).validate(props); } + if (props.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)); + } + if (props.containsKey(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)); + } + } + + private void checkDeserializer(String deserializer) { + try { + Class deserClass = Class.forName(deserializer); + if (!Deserializer.class.isAssignableFrom(deserClass)) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s is not a subclass of %s", + deserializer, Deserializer.class.getName())); + } + + // Get the generic type information + Type[] interfaces = deserClass.getGenericInterfaces(); + for (Type iface : interfaces) { + if (iface instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) iface; + Type rawType = parameterizedType.getRawType(); + + // Check if it's Deserializer + if (rawType == Deserializer.class) { + Type[] typeArguments = parameterizedType.getActualTypeArguments(); + if (typeArguments.length != 1 || typeArguments[0] != byte[].class) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s does not deserialize byte[]", + deserializer)); + } + } + } + } + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException( + String.format("Deserializer class %s not found", deserializer), e); + } } private boolean offsetCommitEnabledManually() { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java index 248014c8a..f96cd3ea0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/KafkaSourceOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import java.time.Duration; import java.util.Properties; import java.util.function.Function; @@ -38,10 +39,11 @@ public class KafkaSourceOptions { public static final ConfigOption PARTITION_DISCOVERY_INTERVAL_MS = ConfigOptions.key("partition.discovery.interval.ms") .longType() - .noDefaultValue() + .defaultValue(Duration.ofMinutes(5).toMillis()) .withDescription( "The interval in milliseconds for the Kafka source to discover " - + "the new partitions. A non-positive value disables the partition discovery."); + + "the new partitions. A non-positive value disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); public static final ConfigOption REGISTER_KAFKA_CONSUMER_METRICS = ConfigOptions.key("register.consumer.metrics") diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java new file mode 100644 index 000000000..b7d115386 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/AssignmentStatus.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; + +/** status of partition assignment. */ +@Internal +public enum AssignmentStatus { + + /** Partitions that have been assigned to readers. */ + ASSIGNED(0), + /** + * The partitions that have been discovered during initialization but not assigned to readers + * yet. + */ + UNASSIGNED_INITIAL(1); + private final int statusCode; + + AssignmentStatus(int statusCode) { + this.statusCode = statusCode; + } + + public int getStatusCode() { + return statusCode; + } + + public static AssignmentStatus ofStatusCode(int statusCode) { + for (AssignmentStatus statusEnum : AssignmentStatus.values()) { + if (statusEnum.getStatusCode() == statusCode) { + return statusEnum; + } + } + throw new IllegalArgumentException("statusCode is invalid."); + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java index 04604abe1..66ceeeb8a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumState.java @@ -22,18 +22,73 @@ import org.apache.kafka.common.TopicPartition; +import java.util.HashSet; import java.util.Set; +import java.util.stream.Collectors; /** The state of Kafka source enumerator. */ @Internal public class KafkaSourceEnumState { - private final Set assignedPartitions; + /** Partitions with status: ASSIGNED or UNASSIGNED_INITIAL. */ + private final Set partitions; + /** + * this flag will be marked as true if initial partitions are discovered after enumerator + * starts. + */ + private final boolean initialDiscoveryFinished; - KafkaSourceEnumState(Set assignedPartitions) { - this.assignedPartitions = assignedPartitions; + public KafkaSourceEnumState( + Set partitions, boolean initialDiscoveryFinished) { + this.partitions = partitions; + this.initialDiscoveryFinished = initialDiscoveryFinished; + } + + public KafkaSourceEnumState( + Set assignPartitions, + Set unassignedInitialPartitions, + boolean initialDiscoveryFinished) { + this.partitions = new HashSet<>(); + partitions.addAll( + assignPartitions.stream() + .map( + topicPartition -> + new TopicPartitionAndAssignmentStatus( + topicPartition, AssignmentStatus.ASSIGNED)) + .collect(Collectors.toSet())); + partitions.addAll( + unassignedInitialPartitions.stream() + .map( + topicPartition -> + new TopicPartitionAndAssignmentStatus( + topicPartition, + AssignmentStatus.UNASSIGNED_INITIAL)) + .collect(Collectors.toSet())); + this.initialDiscoveryFinished = initialDiscoveryFinished; + } + + public Set partitions() { + return partitions; } public Set assignedPartitions() { - return assignedPartitions; + return filterPartitionsByAssignmentStatus(AssignmentStatus.ASSIGNED); + } + + public Set unassignedInitialPartitions() { + return filterPartitionsByAssignmentStatus(AssignmentStatus.UNASSIGNED_INITIAL); + } + + public boolean initialDiscoveryFinished() { + return initialDiscoveryFinished; + } + + private Set filterPartitionsByAssignmentStatus( + AssignmentStatus assignmentStatus) { + return partitions.stream() + .filter( + partitionWithStatus -> + partitionWithStatus.assignmentStatus().equals(assignmentStatus)) + .map(TopicPartitionAndAssignmentStatus::topicPartition) + .collect(Collectors.toSet()); } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java index 7428f4205..f8dc17deb 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.kafka.source.enumerator; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.connector.base.source.utils.SerdeUtils; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; @@ -44,10 +45,20 @@ public class KafkaSourceEnumStateSerializer implements SimpleVersionedSerializer { + /** + * state of VERSION_0 contains splitAssignments, which is a mapping from subtask ids to lists of + * assigned splits. + */ private static final int VERSION_0 = 0; + /** state of VERSION_1 only contains assignedPartitions, which is a list of assigned splits. */ private static final int VERSION_1 = 1; + /** + * state of VERSION_2 contains initialDiscoveryFinished and partitions with different assignment + * status. + */ + private static final int VERSION_2 = 2; - private static final int CURRENT_VERSION = VERSION_1; + private static final int CURRENT_VERSION = VERSION_2; @Override public int getVersion() { @@ -56,69 +67,112 @@ public int getVersion() { @Override public byte[] serialize(KafkaSourceEnumState enumState) throws IOException { - return serializeTopicPartitions(enumState.assignedPartitions()); + Set partitions = enumState.partitions(); + boolean initialDiscoveryFinished = enumState.initialDiscoveryFinished(); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + out.writeInt(partitions.size()); + for (TopicPartitionAndAssignmentStatus topicPartitionAndAssignmentStatus : partitions) { + out.writeUTF(topicPartitionAndAssignmentStatus.topicPartition().topic()); + out.writeInt(topicPartitionAndAssignmentStatus.topicPartition().partition()); + out.writeInt(topicPartitionAndAssignmentStatus.assignmentStatus().getStatusCode()); + } + out.writeBoolean(initialDiscoveryFinished); + out.flush(); + return baos.toByteArray(); + } } @Override public KafkaSourceEnumState deserialize(int version, byte[] serialized) throws IOException { - if (version == CURRENT_VERSION) { - final Set assignedPartitions = deserializeTopicPartitions(serialized); - return new KafkaSourceEnumState(assignedPartitions); + switch (version) { + case CURRENT_VERSION: + return deserializeTopicPartitionAndAssignmentStatus(serialized); + case VERSION_1: + return deserializeAssignedTopicPartitions(serialized); + case VERSION_0: + Map> currentPartitionAssignment = + SerdeUtils.deserializeSplitAssignments( + serialized, new KafkaPartitionSplitSerializer(), HashSet::new); + Set currentAssignedSplits = new HashSet<>(); + currentPartitionAssignment.forEach( + (reader, splits) -> + splits.forEach( + split -> + currentAssignedSplits.add( + split.getTopicPartition()))); + return new KafkaSourceEnumState(currentAssignedSplits, new HashSet<>(), true); + default: + throw new IOException( + String.format( + "The bytes are serialized with version %d, " + + "while this deserializer only supports version up to %d", + version, CURRENT_VERSION)); } - - // Backward compatibility - if (version == VERSION_0) { - Map> currentPartitionAssignment = - SerdeUtils.deserializeSplitAssignments( - serialized, new KafkaPartitionSplitSerializer(), HashSet::new); - Set currentAssignedSplits = new HashSet<>(); - currentPartitionAssignment.forEach( - (reader, splits) -> - splits.forEach( - split -> currentAssignedSplits.add(split.getTopicPartition()))); - return new KafkaSourceEnumState(currentAssignedSplits); - } - - throw new IOException( - String.format( - "The bytes are serialized with version %d, " - + "while this deserializer only supports version up to %d", - version, CURRENT_VERSION)); } - private static byte[] serializeTopicPartitions(Collection topicPartitions) - throws IOException { - try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(baos)) { + private static KafkaSourceEnumState deserializeAssignedTopicPartitions( + byte[] serializedTopicPartitions) throws IOException { + try (ByteArrayInputStream bais = new ByteArrayInputStream(serializedTopicPartitions); + DataInputStream in = new DataInputStream(bais)) { - out.writeInt(topicPartitions.size()); - for (TopicPartition tp : topicPartitions) { - out.writeUTF(tp.topic()); - out.writeInt(tp.partition()); + final int numPartitions = in.readInt(); + Set partitions = new HashSet<>(numPartitions); + for (int i = 0; i < numPartitions; i++) { + final String topic = in.readUTF(); + final int partition = in.readInt(); + partitions.add( + new TopicPartitionAndAssignmentStatus( + new TopicPartition(topic, partition), AssignmentStatus.ASSIGNED)); } - out.flush(); - - return baos.toByteArray(); + if (in.available() > 0) { + throw new IOException("Unexpected trailing bytes in serialized topic partitions"); + } + return new KafkaSourceEnumState(partitions, true); } } - private static Set deserializeTopicPartitions(byte[] serializedTopicPartitions) - throws IOException { - try (ByteArrayInputStream bais = new ByteArrayInputStream(serializedTopicPartitions); + private static KafkaSourceEnumState deserializeTopicPartitionAndAssignmentStatus( + byte[] serialized) throws IOException { + + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); DataInputStream in = new DataInputStream(bais)) { final int numPartitions = in.readInt(); - Set topicPartitions = new HashSet<>(numPartitions); + Set partitions = new HashSet<>(numPartitions); + for (int i = 0; i < numPartitions; i++) { final String topic = in.readUTF(); final int partition = in.readInt(); - topicPartitions.add(new TopicPartition(topic, partition)); + final int statusCode = in.readInt(); + partitions.add( + new TopicPartitionAndAssignmentStatus( + new TopicPartition(topic, partition), + AssignmentStatus.ofStatusCode(statusCode))); } + final boolean initialDiscoveryFinished = in.readBoolean(); if (in.available() > 0) { throw new IOException("Unexpected trailing bytes in serialized topic partitions"); } - return topicPartitions; + return new KafkaSourceEnumState(partitions, initialDiscoveryFinished); + } + } + + @VisibleForTesting + public static byte[] serializeTopicPartitions(Collection topicPartitions) + throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(baos)) { + + out.writeInt(topicPartitions.size()); + for (TopicPartition tp : topicPartitions) { + out.writeUTF(tp.topic()); + out.writeInt(tp.partition()); + } + out.flush(); + + return baos.toByteArray(); } } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java index 137f42043..10025fa2a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumerator.java @@ -65,6 +65,7 @@ public class KafkaSourceEnumerator private final KafkaSubscriber subscriber; private final OffsetsInitializer startingOffsetInitializer; private final OffsetsInitializer stoppingOffsetInitializer; + private final OffsetsInitializer newDiscoveryOffsetsInitializer; private final Properties properties; private final long partitionDiscoveryIntervalMs; private final SplitEnumeratorContext context; @@ -73,6 +74,12 @@ public class KafkaSourceEnumerator /** Partitions that have been assigned to readers. */ private final Set assignedPartitions; + /** + * The partitions that have been discovered during initialization but not assigned to readers + * yet. + */ + private final Set unassignedInitialPartitions; + /** * The discovered and initialized partition splits that are waiting for owner reader to be * ready. @@ -88,6 +95,8 @@ public class KafkaSourceEnumerator // This flag will be marked as true if periodically partition discovery is disabled AND the // initializing partition discovery has finished. private boolean noMoreNewPartitionSplits = false; + // this flag will be marked as true if initial partitions are discovered after enumerator starts + private boolean initialDiscoveryFinished; public KafkaSourceEnumerator( KafkaSubscriber subscriber, @@ -103,7 +112,7 @@ public KafkaSourceEnumerator( properties, context, boundedness, - Collections.emptySet()); + new KafkaSourceEnumState(Collections.emptySet(), false)); } public KafkaSourceEnumerator( @@ -113,15 +122,16 @@ public KafkaSourceEnumerator( Properties properties, SplitEnumeratorContext context, Boundedness boundedness, - Set assignedPartitions) { + KafkaSourceEnumState kafkaSourceEnumState) { this.subscriber = subscriber; this.startingOffsetInitializer = startingOffsetInitializer; this.stoppingOffsetInitializer = stoppingOffsetInitializer; + this.newDiscoveryOffsetsInitializer = OffsetsInitializer.earliest(); this.properties = properties; this.context = context; this.boundedness = boundedness; - this.assignedPartitions = new HashSet<>(assignedPartitions); + this.assignedPartitions = new HashSet<>(kafkaSourceEnumState.assignedPartitions()); this.pendingPartitionSplitAssignment = new HashMap<>(); this.partitionDiscoveryIntervalMs = KafkaSourceOptions.getOption( @@ -129,6 +139,9 @@ public KafkaSourceEnumerator( KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS, Long::parseLong); this.consumerGroupId = properties.getProperty(ConsumerConfig.GROUP_ID_CONFIG); + this.unassignedInitialPartitions = + new HashSet<>(kafkaSourceEnumState.unassignedInitialPartitions()); + this.initialDiscoveryFinished = kafkaSourceEnumState.initialDiscoveryFinished(); } /** @@ -195,7 +208,8 @@ public void addReader(int subtaskId) { @Override public KafkaSourceEnumState snapshotState(long checkpointId) throws Exception { - return new KafkaSourceEnumState(assignedPartitions); + return new KafkaSourceEnumState( + assignedPartitions, unassignedInitialPartitions, initialDiscoveryFinished); } @Override @@ -234,6 +248,12 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa throw new FlinkRuntimeException( "Failed to list subscribed topic partitions due to ", t); } + + if (!initialDiscoveryFinished) { + unassignedInitialPartitions.addAll(fetchedPartitions); + initialDiscoveryFinished = true; + } + final PartitionChange partitionChange = getPartitionChange(fetchedPartitions); if (partitionChange.isEmpty()) { return; @@ -266,10 +286,18 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa private PartitionSplitChange initializePartitionSplits(PartitionChange partitionChange) { Set newPartitions = Collections.unmodifiableSet(partitionChange.getNewPartitions()); + OffsetsInitializer.PartitionOffsetsRetriever offsetsRetriever = getOffsetsRetriever(); + // initial partitions use OffsetsInitializer specified by the user while new partitions use + // EARLIEST + Map startingOffsets = new HashMap<>(); + startingOffsets.putAll( + newDiscoveryOffsetsInitializer.getPartitionOffsets( + newPartitions, offsetsRetriever)); + startingOffsets.putAll( + startingOffsetInitializer.getPartitionOffsets( + unassignedInitialPartitions, offsetsRetriever)); - Map startingOffsets = - startingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); Map stoppingOffsets = stoppingOffsetInitializer.getPartitionOffsets(newPartitions, offsetsRetriever); @@ -344,7 +372,10 @@ private void assignPendingPartitionSplits(Set pendingReaders) { // Mark pending partitions as already assigned pendingAssignmentForReader.forEach( - split -> assignedPartitions.add(split.getTopicPartition())); + split -> { + assignedPartitions.add(split.getTopicPartition()); + unassignedInitialPartitions.remove(split.getTopicPartition()); + }); } } @@ -539,9 +570,9 @@ public Map committedOffsets(Collection par * the beginning offset, end offset as well as the offset matching a timestamp in * partitions. * - * @see KafkaAdminClient#listOffsets(Map) * @param topicPartitionOffsets The mapping from partition to the OffsetSpec to look up. * @return The list offsets result. + * @see KafkaAdminClient#listOffsets(Map) */ private Map listOffsets( Map topicPartitionOffsets) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java new file mode 100644 index 000000000..2caed99b7 --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/TopicPartitionAndAssignmentStatus.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.source.enumerator; + +import org.apache.flink.annotation.Internal; + +import org.apache.kafka.common.TopicPartition; + +/** Kafka partition with assign status. */ +@Internal +public class TopicPartitionAndAssignmentStatus { + private final TopicPartition topicPartition; + private final AssignmentStatus assignmentStatus; + + public TopicPartitionAndAssignmentStatus( + TopicPartition topicPartition, AssignmentStatus assignStatus) { + this.topicPartition = topicPartition; + this.assignmentStatus = assignStatus; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + + public AssignmentStatus assignmentStatus() { + return assignmentStatus; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java new file mode 100644 index 000000000..b6c95a65e --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/LatestOffsetsInitializer.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.source.enumerator.initializer; + +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Map; + +/** + * An implementation of {@link OffsetsInitializer} to initialize the offsets based on a + * latest-offset. + * + *

      Package private and should be instantiated via {@link OffsetsInitializer}. + */ +class LatestOffsetsInitializer implements OffsetsInitializer { + private static final long serialVersionUID = 3014700244733286989L; + + @Override + public Map getPartitionOffsets( + Collection partitions, + PartitionOffsetsRetriever partitionOffsetsRetriever) { + return partitionOffsetsRetriever.endOffsets(partitions); + } + + @Override + public OffsetResetStrategy getAutoOffsetResetStrategy() { + return OffsetResetStrategy.LATEST; + } +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java index db682c6b0..0f0c5d25b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializer.java @@ -155,8 +155,7 @@ static OffsetsInitializer earliest() { * @return an {@link OffsetsInitializer} which initializes the offsets to the latest offsets. */ static OffsetsInitializer latest() { - return new ReaderHandledOffsetsInitializer( - KafkaPartitionSplit.LATEST_OFFSET, OffsetResetStrategy.LATEST); + return new LatestOffsetsInitializer(); } /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java index 026320d93..42abd5778 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/ReaderHandledOffsetsInitializer.java @@ -32,9 +32,9 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * A initializer that initialize the partitions to the earliest / latest / last-committed offsets. - * The offsets initialization are taken care of by the {@code KafkaPartitionSplitReader} instead of - * by the {@code KafkaSourceEnumerator}. + * A initializer that initialize the partitions to the earliest / last-committed offsets. The + * offsets initialization are taken care of by the {@code KafkaPartitionSplitReader} instead of by + * the {@code KafkaSourceEnumerator}. * *

      Package private and should be instantiated via {@link OffsetsInitializer}. */ @@ -46,8 +46,7 @@ class ReaderHandledOffsetsInitializer implements OffsetsInitializer, OffsetsInit /** * The only valid value for startingOffset is following. {@link * KafkaPartitionSplit#EARLIEST_OFFSET EARLIEST_OFFSET}, {@link - * KafkaPartitionSplit#LATEST_OFFSET LATEST_OFFSET}, {@link KafkaPartitionSplit#COMMITTED_OFFSET - * COMMITTED_OFFSET} + * KafkaPartitionSplit#COMMITTED_OFFSET COMMITTED_OFFSET} */ ReaderHandledOffsetsInitializer(long startingOffset, OffsetResetStrategy offsetResetStrategy) { this.startingOffset = startingOffset; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java index 5766a5fc1..e402435bd 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/SpecifiedOffsetsInitializer.java @@ -38,6 +38,11 @@ * An implementation of {@link OffsetsInitializer} which initializes the offsets of the partition * according to the user specified offsets. * + *

      Use specified offsets for specified partitions while use commit offsets or offsetResetStrategy + * for unspecified partitions. Specified partition's offset should be less than its latest offset, + * otherwise it will start from the offsetResetStrategy. The default value of offsetResetStrategy is + * earliest. + * *

      Package private and should be instantiated via {@link OffsetsInitializer}. */ class SpecifiedOffsetsInitializer implements OffsetsInitializer, OffsetsInitializerValidator { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java index f411307f6..0ad53c9b1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/initializer/TimestampOffsetsInitializer.java @@ -28,6 +28,8 @@ /** * An implementation of {@link OffsetsInitializer} to initialize the offsets based on a timestamp. + * If the message meeting the requirement of the timestamp have not been produced to Kafka yet, just + * use the latest offset. * *

      Package private and should be instantiated via {@link OffsetsInitializer}. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java index 1b819fb23..37de884af 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriber.java @@ -39,6 +39,10 @@ * *

      The KafkaSubscriber provides a unified interface for the Kafka source to support all these * three types of subscribing mode. + * + *

      When implementing a subscriber, {@link + * org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider} can be implemented to + * provide lineage metadata with source topics. */ @PublicEvolving public interface KafkaSubscriber extends Serializable { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java index 404ffaef8..72e7f64d0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberUtils.java @@ -23,6 +23,8 @@ import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; /** The base implementations of {@link KafkaSubscriber}. */ class KafkaSubscriberUtils { @@ -38,6 +40,22 @@ static Map getAllTopicMetadata(AdminClient adminClient } } + static Map getTopicMetadata( + AdminClient adminClient, Pattern topicPattern) { + try { + Set allTopicNames = adminClient.listTopics().names().get(); + Set matchedTopicNames = + allTopicNames.stream() + .filter(name -> topicPattern.matcher(name).matches()) + .collect(Collectors.toSet()); + return getTopicMetadata(adminClient, matchedTopicNames); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to get metadata for %s topics.", topicPattern.pattern()), + e); + } + } + static Map getTopicMetadata( AdminClient adminClient, Set topicNames) { try { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java index 3423b0f90..9cd50fb20 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/PartitionSetSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -26,13 +29,14 @@ import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; /** A subscriber for a partition set. */ -class PartitionSetSubscriber implements KafkaSubscriber { +class PartitionSetSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = 390970375272146036L; private static final Logger LOG = LoggerFactory.getLogger(PartitionSetSubscriber.class); private final Set subscribedPartitions; @@ -73,4 +77,14 @@ && partitionExistsInTopic( private boolean partitionExistsInTopic(TopicPartition partition, TopicDescription topic) { return topic.partitions().size() > partition.partition(); } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + subscribedPartitions.stream() + .map(TopicPartition::topic) + .distinct() + .collect(Collectors.toList()))); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java index b2ad844ab..e86ade0fa 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicListSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -28,6 +31,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; @@ -36,7 +40,7 @@ * A subscriber to a fixed list of topics. The subscribed topics must have existed in the Kafka * cluster, otherwise an exception will be thrown. */ -class TopicListSubscriber implements KafkaSubscriber { +class TopicListSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = -6917603843104947866L; private static final Logger LOG = LoggerFactory.getLogger(TopicListSubscriber.class); private final List topics; @@ -60,4 +64,9 @@ public Set getSubscribedTopicPartitions(AdminClient adminClient) return subscribedPartitions; } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of(DefaultKafkaDatasetIdentifier.ofTopics(topics)); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java index 2a9a75331..208959e27 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/TopicPatternSubscriber.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; + import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.TopicPartition; @@ -27,13 +30,14 @@ import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.regex.Pattern; -import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getAllTopicMetadata; +import static org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriberUtils.getTopicMetadata; /** A subscriber to a topic pattern. */ -class TopicPatternSubscriber implements KafkaSubscriber { +class TopicPatternSubscriber implements KafkaSubscriber, KafkaDatasetIdentifierProvider { private static final long serialVersionUID = -7471048577725467797L; private static final Logger LOG = LoggerFactory.getLogger(TopicPatternSubscriber.class); private final Pattern topicPattern; @@ -44,22 +48,25 @@ class TopicPatternSubscriber implements KafkaSubscriber { @Override public Set getSubscribedTopicPartitions(AdminClient adminClient) { - LOG.debug("Fetching descriptions for all topics on Kafka cluster"); - final Map allTopicMetadata = getAllTopicMetadata(adminClient); + LOG.debug("Fetching descriptions for {} topics on Kafka cluster", topicPattern.pattern()); + final Map matchedTopicMetadata = + getTopicMetadata(adminClient, topicPattern); Set subscribedTopicPartitions = new HashSet<>(); - allTopicMetadata.forEach( + matchedTopicMetadata.forEach( (topicName, topicDescription) -> { - if (topicPattern.matcher(topicName).matches()) { - for (TopicPartitionInfo partition : topicDescription.partitions()) { - subscribedTopicPartitions.add( - new TopicPartition( - topicDescription.name(), partition.partition())); - } + for (TopicPartitionInfo partition : topicDescription.partitions()) { + subscribedTopicPartitions.add( + new TopicPartition(topicDescription.name(), partition.partition())); } }); return subscribedTopicPartitions; } + + @Override + public Optional getDatasetIdentifier() { + return Optional.of(DefaultKafkaDatasetIdentifier.ofPattern(topicPattern)); + } } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java index f52940c4b..23956f5d5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java @@ -80,11 +80,20 @@ public KafkaPartitionSplitReader( Properties props, SourceReaderContext context, KafkaSourceReaderMetrics kafkaSourceReaderMetrics) { + this(props, context, kafkaSourceReaderMetrics, null); + } + + public KafkaPartitionSplitReader( + Properties props, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics, + String rackIdSupplier) { this.subtaskId = context.getIndexOfSubtask(); this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics; Properties consumerProps = new Properties(); consumerProps.putAll(props); consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, createConsumerClientId(props)); + setConsumerClientRack(consumerProps, rackIdSupplier); this.consumer = new KafkaConsumer<>(consumerProps); this.stoppingOffsets = new HashMap<>(); this.groupId = consumerProps.getProperty(ConsumerConfig.GROUP_ID_CONFIG); @@ -113,32 +122,32 @@ public RecordsWithSplitIds> fetch() throws IOExce KafkaPartitionSplitRecords recordsBySplits = new KafkaPartitionSplitRecords(consumerRecords, kafkaSourceReaderMetrics); List finishedPartitions = new ArrayList<>(); - for (TopicPartition tp : consumerRecords.partitions()) { + for (TopicPartition tp : consumer.assignment()) { long stoppingOffset = getStoppingOffset(tp); - final List> recordsFromPartition = - consumerRecords.records(tp); - - if (recordsFromPartition.size() > 0) { - final ConsumerRecord lastRecord = - recordsFromPartition.get(recordsFromPartition.size() - 1); - - // After processing a record with offset of "stoppingOffset - 1", the split reader - // should not continue fetching because the record with stoppingOffset may not - // exist. Keep polling will just block forever. - if (lastRecord.offset() >= stoppingOffset - 1) { - recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); - finishSplitAtRecord( - tp, - stoppingOffset, - lastRecord.offset(), - finishedPartitions, - recordsBySplits); - } + long consumerPosition = consumer.position(tp); + // Stop fetching when the consumer's position reaches the stoppingOffset. + // Control messages may follow the last record; therefore, using the last record's + // offset as a stopping condition could result in indefinite blocking. + if (consumerPosition >= stoppingOffset) { + LOG.debug( + "Position of {}: {}, has reached stopping offset: {}", + tp, + consumerPosition, + stoppingOffset); + recordsBySplits.setPartitionStoppingOffset(tp, stoppingOffset); + finishSplitAtRecord( + tp, stoppingOffset, consumerPosition, finishedPartitions, recordsBySplits); } - // Track this partition's record lag if it never appears before - kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp); } + // Only track non-empty partition's record lag if it never appears before + consumerRecords + .partitions() + .forEach( + trackTp -> { + kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, trackTp); + }); + markEmptySplitsAsFinished(recordsBySplits); // Unassign the partitions that has finished. @@ -256,6 +265,20 @@ KafkaConsumer consumer() { // --------------- private helper method ---------------------- + /** + * This Method performs Null and empty Rack Id validation and sets the rack id to the + * client.rack Consumer Config. + * + * @param consumerProps Consumer Property. + * @param rackId Rack Id's. + */ + @VisibleForTesting + void setConsumerClientRack(Properties consumerProps, String rackId) { + if (rackId != null && !rackId.isEmpty()) { + consumerProps.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, rackId); + } + } + private void parseStartingOffsets( KafkaPartitionSplit split, List partitionsStartingFromEarliest, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java index e8fe3d776..cb85fc89d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReader.java @@ -38,7 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -133,9 +132,13 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { Map committedPartitions = offsetsToCommit.get(checkpointId); if (committedPartitions == null) { - LOG.debug( - "Offsets for checkpoint {} either do not exist or have already been committed.", - checkpointId); + LOG.debug("Offsets for checkpoint {} have already been committed.", checkpointId); + return; + } + + if (committedPartitions.isEmpty()) { + LOG.debug("There are no offsets to commit for checkpoint {}.", checkpointId); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); return; } @@ -168,14 +171,17 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { entry -> committedPartitions.containsKey( entry.getKey())); - while (!offsetsToCommit.isEmpty() - && offsetsToCommit.firstKey() <= checkpointId) { - offsetsToCommit.remove(offsetsToCommit.firstKey()); - } + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); } }); } + private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { + while (!offsetsToCommit.isEmpty() && offsetsToCommit.firstKey() <= checkpointId) { + offsetsToCommit.remove(offsetsToCommit.firstKey()); + } + } + @Override protected KafkaPartitionSplitState initializedState(KafkaPartitionSplit split) { return new KafkaPartitionSplitState(split); @@ -186,12 +192,6 @@ protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitSta return splitState.toKafkaPartitionSplit(); } - @Override - public void pauseOrResumeSplits( - Collection splitsToPause, Collection splitsToResume) { - splitFetcherManager.pauseOrResumeSplits(splitsToPause, splitsToResume); - } - // ------------------------ @VisibleForTesting diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java index 94197e347..1cc7dde79 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaDeserializationSchemaWrapper.java @@ -33,7 +33,9 @@ * ConsumerRecord ConsumerRecords}. * * @param the type of the deserialized records. + * @deprecated Remove with @{@link KafkaDeserializationSchema} */ +@Deprecated class KafkaDeserializationSchemaWrapper implements KafkaRecordDeserializationSchema { private static final long serialVersionUID = 1L; private final KafkaDeserializationSchema kafkaDeserializationSchema; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java index 6ad6607c9..91d1f3439 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchema.java @@ -71,7 +71,9 @@ default void open(DeserializationSchema.InitializationContext context) throws Ex * @param the return type of the deserialized record. * @return A {@link KafkaRecordDeserializationSchema} that uses the given {@link * KafkaDeserializationSchema} to deserialize the {@link ConsumerRecord ConsumerRecords}. + * @deprecated Will be removed with {@link KafkaDeserializationSchema}. */ + @Deprecated static KafkaRecordDeserializationSchema of( KafkaDeserializationSchema kafkaDeserializationSchema) { return new KafkaDeserializationSchemaWrapper<>(kafkaDeserializationSchema); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java index 8c2a1fd1f..7c04600d1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplit.java @@ -35,7 +35,8 @@ public class KafkaPartitionSplit implements SourceSplit { public static final long NO_STOPPING_OFFSET = Long.MIN_VALUE; // Indicating the split should consume from the latest. - public static final long LATEST_OFFSET = -1; + // @deprecated Only be used for compatibility with the history state, see FLINK-28303 + @Deprecated public static final long LATEST_OFFSET = -1; // Indicating the split should consume from the earliest. public static final long EARLIEST_OFFSET = -2; // Indicating the split should consume from the last committed offset. diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java new file mode 100644 index 000000000..c9301c79c --- /dev/null +++ b/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/util/JacksonMapperFactory.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.util; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; + +/** Factory for Jackson mappers. */ +public final class JacksonMapperFactory { + + public static ObjectMapper createObjectMapper() { + final ObjectMapper objectMapper = new ObjectMapper(); + registerModules(objectMapper); + return objectMapper; + } + + public static ObjectMapper createObjectMapper(JsonFactory jsonFactory) { + final ObjectMapper objectMapper = new ObjectMapper(jsonFactory); + registerModules(objectMapper); + return objectMapper; + } + + private static void registerModules(ObjectMapper mapper) { + mapper.registerModule(new JavaTimeModule()) + .registerModule(new Jdk8Module().configureAbsentsAsNulls(true)) + .disable(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) + .disable(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS); + } + + private JacksonMapperFactory() {} +} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index f291b05bc..7a85b434e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -90,6 +90,7 @@ * @param The type of records produced by this data source */ @Internal +@Deprecated public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFunction implements CheckpointListener, ResultTypeQueryable, CheckpointedFunction { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java index c11f1b176..3c1ae27e3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaErrorCode.java @@ -19,8 +19,14 @@ import org.apache.flink.annotation.PublicEvolving; -/** Error codes used in {@link FlinkKafkaException}. */ +/** + * Error codes used in {@link FlinkKafkaException}. + * + * @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link + * org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}. + */ @PublicEvolving +@Deprecated public enum FlinkKafkaErrorCode { PRODUCERS_POOL_EMPTY, EXTERNAL_ERROR diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java index 77d023130..65b654c64 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaException.java @@ -20,8 +20,14 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.util.FlinkException; -/** Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. */ +/** + * Exception used by {@link FlinkKafkaProducer} and {@link FlinkKafkaConsumer}. + * + * @deprecated Will be removed with {@link FlinkKafkaProducer} and {@link + * org.apache.flink.streaming.connectors.kafka.shuffle.FlinkKafkaShuffle}. + */ @PublicEvolving +@Deprecated public class FlinkKafkaException extends FlinkException { private static final long serialVersionUID = 920269130311214200L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index ca81cc8e3..4b39749d9 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -29,10 +29,12 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.metrics.MetricGroup; @@ -49,11 +51,8 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.TemporaryClassLoaderContext; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.commons.lang3.StringUtils; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; @@ -72,6 +71,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.io.ObjectInputStream; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -93,10 +93,10 @@ /** * Flink Sink to produce data into a Kafka topic. By default producer will use {@link - * FlinkKafkaProducer.Semantic#AT_LEAST_ONCE} semantic. Before using {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE} please refer to Flink's Kafka connector documentation. + * Semantic#AT_LEAST_ONCE} semantic. Before using {@link Semantic#EXACTLY_ONCE} please refer to + * Flink's Kafka connector documentation. * - * @deprecated Please use {@link org.apache.flink.connector.kafka.sink.KafkaSink}. + * @deprecated Please use {@link KafkaSink}. */ @Deprecated @PublicEvolving @@ -175,10 +175,7 @@ public enum Semantic { */ public static final int SAFE_SCALE_DOWN_FACTOR = 5; - /** - * Default number of KafkaProducers in the pool. See {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}. - */ + /** Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}. */ public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5; /** Default value for kafka transaction timeout. */ @@ -193,27 +190,26 @@ public enum Semantic { * NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2. */ @Deprecated - private static final ListStateDescriptor + private static final ListStateDescriptor NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR = new ListStateDescriptor<>( "next-transactional-id-hint", TypeInformation.of(NextTransactionalIdHint.class)); - private static final ListStateDescriptor + private static final ListStateDescriptor NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2 = new ListStateDescriptor<>( "next-transactional-id-hint-v2", new NextTransactionalIdHintSerializer()); /** State for nextTransactionalIdHint. */ - private transient ListState - nextTransactionalIdHintState; + private transient ListState nextTransactionalIdHintState; /** Generator for Transactional IDs. */ private transient TransactionalIdsGenerator transactionalIdsGenerator; /** Hint for picking next transactional id. */ - private transient FlinkKafkaProducer.NextTransactionalIdHint nextTransactionalIdHint; + private transient NextTransactionalIdHint nextTransactionalIdHint; /** User defined properties for the Producer. */ protected final Properties producerConfig; @@ -258,7 +254,7 @@ public enum Semantic { private boolean logFailuresOnly; /** Semantic chosen for this instance. */ - protected FlinkKafkaProducer.Semantic semantic; + protected Semantic semantic; // -------------------------------- Runtime fields ------------------------------------------ @@ -362,17 +358,16 @@ public FlinkKafkaProducer( * @param customPartitioner A serializable partitioner for assigning messages to Kafka * partitions. If a partitioner is not provided, records will be distributed to Kafka * partitions in a round-robin fashion. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ public FlinkKafkaProducer( String topicId, SerializationSchema serializationSchema, Properties producerConfig, @Nullable FlinkKafkaPartitioner customPartitioner, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( topicId, @@ -401,7 +396,7 @@ public FlinkKafkaProducer( * @param topicId ID of the Kafka topic. * @param serializationSchema User defined serialization schema supporting key/value messages * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -427,7 +422,7 @@ public FlinkKafkaProducer( * @param serializationSchema User defined serialization schema supporting key/value messages * @param producerConfig Properties with the producer configuration. * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -451,17 +446,16 @@ public FlinkKafkaProducer( * @param topicId ID of the Kafka topic. * @param serializationSchema User defined serialization schema supporting key/value messages * @param producerConfig Properties with the producer configuration. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( String topicId, KeyedSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic) { + Semantic semantic) { this( topicId, serializationSchema, @@ -493,7 +487,7 @@ public FlinkKafkaProducer( * keys are {@code null}, then records will be distributed to Kafka partitions in a * round-robin fashion. * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -506,7 +500,7 @@ public FlinkKafkaProducer( serializationSchema, producerConfig, customPartitioner, - FlinkKafkaProducer.Semantic.AT_LEAST_ONCE, + Semantic.AT_LEAST_ONCE, DEFAULT_KAFKA_PRODUCERS_POOL_SIZE); } @@ -531,12 +525,11 @@ public FlinkKafkaProducer( * each record (determined by {@link KeyedSerializationSchema#serializeKey(Object)}). If the * keys are {@code null}, then records will be distributed to Kafka partitions in a * round-robin fashion. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). * @deprecated use {@link #FlinkKafkaProducer(String, KafkaSerializationSchema, Properties, - * FlinkKafkaProducer.Semantic)} + * Semantic)} */ @Deprecated public FlinkKafkaProducer( @@ -544,7 +537,7 @@ public FlinkKafkaProducer( KeyedSerializationSchema serializationSchema, Properties producerConfig, Optional> customPartitioner, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( defaultTopicId, @@ -566,14 +559,13 @@ public FlinkKafkaProducer( * a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). */ public FlinkKafkaProducer( String defaultTopic, KafkaSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic) { + Semantic semantic) { this( defaultTopic, serializationSchema, @@ -592,16 +584,15 @@ public FlinkKafkaProducer( * a kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ public FlinkKafkaProducer( String defaultTopic, KafkaSerializationSchema serializationSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { this( defaultTopic, @@ -636,10 +627,9 @@ public FlinkKafkaProducer( * kafka-consumable byte[] supporting key/value messages * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is * the only required argument. - * @param semantic Defines semantic that will be used by this producer (see {@link - * FlinkKafkaProducer.Semantic}). + * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}). * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link - * FlinkKafkaProducer.Semantic#EXACTLY_ONCE}). + * Semantic#EXACTLY_ONCE}). */ private FlinkKafkaProducer( String defaultTopic, @@ -647,11 +637,9 @@ private FlinkKafkaProducer( FlinkKafkaPartitioner customPartitioner, KafkaSerializationSchema kafkaSchema, Properties producerConfig, - FlinkKafkaProducer.Semantic semantic, + Semantic semantic, int kafkaProducersPoolSize) { - super( - new FlinkKafkaProducer.TransactionStateSerializer(), - new FlinkKafkaProducer.ContextStateSerializer()); + super(new TransactionStateSerializer(), new ContextStateSerializer()); this.defaultTopicId = checkNotNull(defaultTopic, "defaultTopic is null"); @@ -731,7 +719,7 @@ private FlinkKafkaProducer( // Enable transactionTimeoutWarnings to avoid silent data loss // See KAFKA-6119 (affects versions 0.11.0.0 and 0.11.0.1): // The KafkaProducer may not throw an exception if the transaction failed to commit - if (semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (semantic == Semantic.EXACTLY_ONCE) { final long transactionTimeout = getTransactionTimeout(producerConfig); super.setTransactionTimeout(transactionTimeout); super.enableTransactionTimeoutWarnings(0.8); @@ -782,7 +770,7 @@ public void setLogFailuresOnly(boolean logFailuresOnly) { * @throws NullPointerException Thrown, if the transactionalIdPrefix was null. */ public void setTransactionalIdPrefix(String transactionalIdPrefix) { - this.transactionalIdPrefix = Preconditions.checkNotNull(transactionalIdPrefix); + this.transactionalIdPrefix = checkNotNull(transactionalIdPrefix); } /** @@ -855,8 +843,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } @Override - public void invoke( - FlinkKafkaProducer.KafkaTransactionState transaction, IN next, Context context) + public void invoke(KafkaTransactionState transaction, IN next, Context context) throws FlinkKafkaException { checkErroneous(); @@ -931,6 +918,10 @@ public void close() throws FlinkKafkaException { // First close the producer for current transaction. try { final KafkaTransactionState currentTransaction = currentTransaction(); + LOG.error( + "Closing producer for current transaction: {} {}", + currentTransaction, + currentTransaction != null ? currentTransaction.producer : null); if (currentTransaction != null) { // to avoid exceptions on aborting transactions with some pending records flush(currentTransaction); @@ -980,33 +971,27 @@ public void close() throws FlinkKafkaException { // ------------------- Logic for handling checkpoint flushing -------------------------- // @Override - protected FlinkKafkaProducer.KafkaTransactionState beginTransaction() - throws FlinkKafkaException { + protected KafkaTransactionState beginTransaction() throws FlinkKafkaException { switch (semantic) { case EXACTLY_ONCE: FlinkKafkaInternalProducer producer = createTransactionalProducer(); producer.beginTransaction(); - return new FlinkKafkaProducer.KafkaTransactionState( - producer.getTransactionalId(), producer); + return new KafkaTransactionState(producer.getTransactionalId(), producer); case AT_LEAST_ONCE: case NONE: // Do not create new producer on each beginTransaction() if it is not necessary - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = - currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction != null && currentTransaction.producer != null) { - return new FlinkKafkaProducer.KafkaTransactionState( - currentTransaction.producer); + return new KafkaTransactionState(currentTransaction.producer); } - return new FlinkKafkaProducer.KafkaTransactionState( - initNonTransactionalProducer(true)); + return new KafkaTransactionState(initNonTransactionalProducer(true)); default: throw new UnsupportedOperationException("Not implemented semantic"); } } @Override - protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) - throws FlinkKafkaException { + protected void preCommit(KafkaTransactionState transaction) throws FlinkKafkaException { switch (semantic) { case EXACTLY_ONCE: case AT_LEAST_ONCE: @@ -1021,7 +1006,7 @@ protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) } @Override - protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void commit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { try { transaction.producer.commitTransaction(); @@ -1032,7 +1017,7 @@ protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { } @Override - protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void recoverAndCommit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { FlinkKafkaInternalProducer producer = null; try { @@ -1065,7 +1050,7 @@ protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transac } @Override - protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void abort(KafkaTransactionState transaction) { if (transaction.isTransactional()) { transaction.producer.abortTransaction(); recycleTransactionalProducer(transaction.producer); @@ -1073,7 +1058,7 @@ protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) { } @Override - protected void recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction) { + protected void recoverAndAbort(KafkaTransactionState transaction) { if (transaction.isTransactional()) { FlinkKafkaInternalProducer producer = null; try { @@ -1101,8 +1086,7 @@ protected void acknowledgeMessage() { * * @param transaction */ - private void flush(FlinkKafkaProducer.KafkaTransactionState transaction) - throws FlinkKafkaException { + private void flush(KafkaTransactionState transaction) throws FlinkKafkaException { if (transaction.producer != null) { transaction.producer.flush(); } @@ -1124,8 +1108,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { // To avoid duplication only first subtask keeps track of next transactional id hint. // Otherwise all of the // subtasks would write exactly same information. - if (getRuntimeContext().getIndexOfThisSubtask() == 0 - && semantic == FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (getRuntimeContext().getIndexOfThisSubtask() == 0 && semantic == Semantic.EXACTLY_ONCE) { checkState( nextTransactionalIdHint != null, "nextTransactionalIdHint must be set for EXACTLY_ONCE"); @@ -1143,7 +1126,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } nextTransactionalIdHintState.add( - new FlinkKafkaProducer.NextTransactionalIdHint( + new NextTransactionalIdHint( getRuntimeContext().getNumberOfParallelSubtasks(), nextFreeTransactionalId)); } @@ -1151,13 +1134,13 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void initializeState(FunctionInitializationContext context) throws Exception { - if (semantic != FlinkKafkaProducer.Semantic.NONE + if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { LOG.warn( "Using {} semantic, but checkpointing is not enabled. Switching to {} semantic.", semantic, - FlinkKafkaProducer.Semantic.NONE); - semantic = FlinkKafkaProducer.Semantic.NONE; + Semantic.NONE); + semantic = Semantic.NONE; } nextTransactionalIdHintState = @@ -1197,16 +1180,17 @@ public void initializeState(FunctionInitializationContext context) throws Except kafkaProducersPoolSize, SAFE_SCALE_DOWN_FACTOR); - if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + if (semantic != Semantic.EXACTLY_ONCE) { nextTransactionalIdHint = null; } else { - ArrayList transactionalIdHints = - Lists.newArrayList(nextTransactionalIdHintState.get()); + List transactionalIdHints = new ArrayList<>(); + nextTransactionalIdHintState.get().forEach(transactionalIdHints::add); + if (transactionalIdHints.size() > 1) { throw new IllegalStateException( "There should be at most one next transactional id hint written by the first subtask"); } else if (transactionalIdHints.size() == 0) { - nextTransactionalIdHint = new FlinkKafkaProducer.NextTransactionalIdHint(0, 0); + nextTransactionalIdHint = new NextTransactionalIdHint(0, 0); // this means that this is either: // (1) the first execution of this application @@ -1223,14 +1207,14 @@ public void initializeState(FunctionInitializationContext context) throws Except } @Override - protected Optional initializeUserContext() { - if (semantic != FlinkKafkaProducer.Semantic.EXACTLY_ONCE) { + protected Optional initializeUserContext() { + if (semantic != Semantic.EXACTLY_ONCE) { return Optional.empty(); } Set transactionalIds = generateNewTransactionalIds(); resetAvailableTransactionalIdsPool(transactionalIds); - return Optional.of(new FlinkKafkaProducer.KafkaTransactionContext(transactionalIds)); + return Optional.of(new KafkaTransactionContext(transactionalIds)); } private Set generateNewTransactionalIds() { @@ -1246,8 +1230,18 @@ private Set generateNewTransactionalIds() { } @Override - protected void finishRecoveringContext( - Collection handledTransactions) { + protected void finishProcessing(@Nullable KafkaTransactionState transaction) { + super.finishProcessing(transaction); + // TwoPhaseCommitSink sets transaction = null on final checkpoint and thus closing will leak + // the producer. For transactional producers, we track the producer in pendingTransactions. + if (transaction != null && !transaction.isTransactional()) { + transaction.producer.flush(); + transaction.producer.close(Duration.ZERO); + } + } + + @Override + protected void finishRecoveringContext(Collection handledTransactions) { cleanUpUserContext(handledTransactions); resetAvailableTransactionalIdsPool(getUserContext().get().transactionalIds); LOG.info("Recovered transactionalIds {}", getUserContext().get().transactionalIds); @@ -1264,8 +1258,7 @@ protected FlinkKafkaInternalProducer createProducer() { * @param handledTransactions transactions which were already committed or aborted and do not * need further handling */ - private void cleanUpUserContext( - Collection handledTransactions) { + private void cleanUpUserContext(Collection handledTransactions) { if (!getUserContext().isPresent()) { return; } @@ -1320,7 +1313,7 @@ private void abortTransactions(final Set transactionalIds) { } int getTransactionCoordinatorId() { - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction == null || currentTransaction.producer == null) { throw new IllegalArgumentException(); } @@ -1329,7 +1322,7 @@ int getTransactionCoordinatorId() { @VisibleForTesting String getTransactionalId() { - final FlinkKafkaProducer.KafkaTransactionState currentTransaction = currentTransaction(); + final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction == null || currentTransaction.producer == null) { throw new IllegalArgumentException(); } @@ -1415,6 +1408,7 @@ private FlinkKafkaInternalProducer initProducer(boolean register } } } + LOG.error("InitProducer {} {}", producerConfig, producer); return producer; } @@ -1430,8 +1424,7 @@ protected void checkErroneous() throws FlinkKafkaException { } } - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); } @@ -1444,8 +1437,9 @@ private void migrateNextTransactionalIdHindState(FunctionInitializationContext c context.getOperatorStateStore() .getUnionListState(NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR_V2); - ArrayList oldTransactionalIdHints = - Lists.newArrayList(oldNextTransactionalIdHintState.get()); + List oldTransactionalIdHints = new ArrayList<>(); + oldNextTransactionalIdHintState.get().forEach(oldTransactionalIdHints::add); + if (!oldTransactionalIdHints.isEmpty()) { nextTransactionalIdHintState.addAll(oldTransactionalIdHints); // clear old state @@ -1561,8 +1555,7 @@ public boolean equals(Object o) { return false; } - FlinkKafkaProducer.KafkaTransactionState that = - (FlinkKafkaProducer.KafkaTransactionState) o; + KafkaTransactionState that = (KafkaTransactionState) o; if (producerId != that.producerId) { return false; @@ -1608,8 +1601,7 @@ public boolean equals(Object o) { return false; } - FlinkKafkaProducer.KafkaTransactionContext that = - (FlinkKafkaProducer.KafkaTransactionContext) o; + KafkaTransactionContext that = (KafkaTransactionContext) o; return transactionalIds.equals(that.transactionalIds); } @@ -1620,14 +1612,11 @@ public int hashCode() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.KafkaTransactionState}. - */ + /** {@link TypeSerializer} for {@link KafkaTransactionState}. */ @VisibleForTesting @Internal public static class TransactionStateSerializer - extends TypeSerializerSingleton { + extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @@ -1637,20 +1626,17 @@ public boolean isImmutableType() { } @Override - public FlinkKafkaProducer.KafkaTransactionState createInstance() { + public KafkaTransactionState createInstance() { return null; } @Override - public FlinkKafkaProducer.KafkaTransactionState copy( - FlinkKafkaProducer.KafkaTransactionState from) { + public KafkaTransactionState copy(KafkaTransactionState from) { return from; } @Override - public FlinkKafkaProducer.KafkaTransactionState copy( - FlinkKafkaProducer.KafkaTransactionState from, - FlinkKafkaProducer.KafkaTransactionState reuse) { + public KafkaTransactionState copy(KafkaTransactionState from, KafkaTransactionState reuse) { return from; } @@ -1660,8 +1646,7 @@ public int getLength() { } @Override - public void serialize( - FlinkKafkaProducer.KafkaTransactionState record, DataOutputView target) + public void serialize(KafkaTransactionState record, DataOutputView target) throws IOException { if (record.transactionalId == null) { target.writeBoolean(false); @@ -1674,21 +1659,18 @@ public void serialize( } @Override - public FlinkKafkaProducer.KafkaTransactionState deserialize(DataInputView source) - throws IOException { + public KafkaTransactionState deserialize(DataInputView source) throws IOException { String transactionalId = null; if (source.readBoolean()) { transactionalId = source.readUTF(); } long producerId = source.readLong(); short epoch = source.readShort(); - return new FlinkKafkaProducer.KafkaTransactionState( - transactionalId, producerId, epoch, null); + return new KafkaTransactionState(transactionalId, producerId, epoch, null); } @Override - public FlinkKafkaProducer.KafkaTransactionState deserialize( - FlinkKafkaProducer.KafkaTransactionState reuse, DataInputView source) + public KafkaTransactionState deserialize(KafkaTransactionState reuse, DataInputView source) throws IOException { return deserialize(source); } @@ -1707,15 +1689,14 @@ public void copy(DataInputView source, DataOutputView target) throws IOException // ----------------------------------------------------------------------------------- @Override - public TypeSerializerSnapshot - snapshotConfiguration() { + public TypeSerializerSnapshot snapshotConfiguration() { return new TransactionStateSerializerSnapshot(); } /** Serializer configuration snapshot for compatibility and format evolution. */ @SuppressWarnings("WeakerAccess") public static final class TransactionStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { + extends SimpleTypeSerializerSnapshot { public TransactionStateSerializerSnapshot() { super(TransactionStateSerializer::new); @@ -1723,14 +1704,11 @@ public TransactionStateSerializerSnapshot() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.KafkaTransactionContext}. - */ + /** {@link TypeSerializer} for {@link KafkaTransactionContext}. */ @VisibleForTesting @Internal public static class ContextStateSerializer - extends TypeSerializerSingleton { + extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @@ -1740,20 +1718,18 @@ public boolean isImmutableType() { } @Override - public FlinkKafkaProducer.KafkaTransactionContext createInstance() { + public KafkaTransactionContext createInstance() { return null; } @Override - public FlinkKafkaProducer.KafkaTransactionContext copy( - FlinkKafkaProducer.KafkaTransactionContext from) { + public KafkaTransactionContext copy(KafkaTransactionContext from) { return from; } @Override - public FlinkKafkaProducer.KafkaTransactionContext copy( - FlinkKafkaProducer.KafkaTransactionContext from, - FlinkKafkaProducer.KafkaTransactionContext reuse) { + public KafkaTransactionContext copy( + KafkaTransactionContext from, KafkaTransactionContext reuse) { return from; } @@ -1763,8 +1739,7 @@ public int getLength() { } @Override - public void serialize( - FlinkKafkaProducer.KafkaTransactionContext record, DataOutputView target) + public void serialize(KafkaTransactionContext record, DataOutputView target) throws IOException { int numIds = record.transactionalIds.size(); target.writeInt(numIds); @@ -1774,20 +1749,18 @@ public void serialize( } @Override - public FlinkKafkaProducer.KafkaTransactionContext deserialize(DataInputView source) - throws IOException { + public KafkaTransactionContext deserialize(DataInputView source) throws IOException { int numIds = source.readInt(); Set ids = new HashSet<>(numIds); for (int i = 0; i < numIds; i++) { ids.add(source.readUTF()); } - return new FlinkKafkaProducer.KafkaTransactionContext(ids); + return new KafkaTransactionContext(ids); } @Override - public FlinkKafkaProducer.KafkaTransactionContext deserialize( - FlinkKafkaProducer.KafkaTransactionContext reuse, DataInputView source) - throws IOException { + public KafkaTransactionContext deserialize( + KafkaTransactionContext reuse, DataInputView source) throws IOException { return deserialize(source); } @@ -1869,10 +1842,7 @@ public int hashCode() { } } - /** - * {@link org.apache.flink.api.common.typeutils.TypeSerializer} for {@link - * FlinkKafkaProducer.NextTransactionalIdHint}. - */ + /** {@link TypeSerializer} for {@link NextTransactionalIdHint}. */ @VisibleForTesting @Internal public static class NextTransactionalIdHintSerializer diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java deleted file mode 100644 index c5b008c1e..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; - -/** - * Compatibility class to make migration possible from the 0.11 connector to the universal one. - * - *

      Problem is that FlinkKafkaProducer (universal) and FlinkKafkaProducer011 have different names - * and they both defined static classes NextTransactionalIdHint, KafkaTransactionState and - * KafkaTransactionContext inside the parent classes. This is causing incompatibility problems since - * for example FlinkKafkaProducer011.KafkaTransactionState and - * FlinkKafkaProducer.KafkaTransactionState are treated as completely incompatible classes, despite - * being identical. - * - *

      This issue is solved by using custom serialization logic: keeping a fake/dummy - * FlinkKafkaProducer011.*Serializer classes in the universal connector (this class), as entry - * points for the deserialization and converting them to FlinkKafkaProducer.*Serializer counter - * parts. After all serialized binary data are exactly the same in all of those cases. - * - *

      For more details check FLINK-11249 and the discussion in the pull requests. - */ -// CHECKSTYLE:OFF: JavadocType -public class FlinkKafkaProducer011 { - public static class NextTransactionalIdHintSerializer { - public static final class NextTransactionalIdHintSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public NextTransactionalIdHintSerializerSnapshot() { - super(FlinkKafkaProducer.NextTransactionalIdHintSerializer::new); - } - } - } - - public static class ContextStateSerializer { - public static final class ContextStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public ContextStateSerializerSnapshot() { - super(FlinkKafkaProducer.ContextStateSerializer::new); - } - } - } - - public static class TransactionStateSerializer { - public static final class TransactionStateSerializerSnapshot - extends SimpleTypeSerializerSnapshot { - public TransactionStateSerializerSnapshot() { - super(FlinkKafkaProducer.TransactionStateSerializer::new); - } - } - } - - public static class NextTransactionalIdHint - extends FlinkKafkaProducer.NextTransactionalIdHint {} -} -// CHECKSTYLE:ON: JavadocType diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java deleted file mode 100644 index 234fd9afa..000000000 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBase.java +++ /dev/null @@ -1,442 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.RuntimeContextInitializationContextAdapters; -import org.apache.flink.api.java.ClosureCleaner; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.util.NetUtils; -import org.apache.flink.util.SerializableObject; - -import org.apache.kafka.clients.producer.Callback; -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.Metric; -import org.apache.kafka.common.MetricName; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import static java.util.Objects.requireNonNull; - -/** - * Flink Sink to produce data into a Kafka topic. - * - *

      Please note that this producer provides at-least-once reliability guarantees when checkpoints - * are enabled and setFlushOnCheckpoint(true) is set. Otherwise, the producer doesn't provide any - * reliability guarantees. - * - * @param Type of the messages to write into Kafka. - */ -@Internal -public abstract class FlinkKafkaProducerBase extends RichSinkFunction - implements CheckpointedFunction { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class); - - private static final long serialVersionUID = 1L; - - /** Configuration key for disabling the metrics reporting. */ - public static final String KEY_DISABLE_METRICS = "flink.disable-metrics"; - - /** User defined properties for the Producer. */ - protected final Properties producerConfig; - - /** The name of the default topic this producer is writing data to. */ - protected final String defaultTopicId; - - /** - * (Serializable) SerializationSchema for turning objects used with Flink into. byte[] for - * Kafka. - */ - protected final KeyedSerializationSchema schema; - - /** User-provided partitioner for assigning an object to a Kafka partition for each topic. */ - protected final FlinkKafkaPartitioner flinkKafkaPartitioner; - - /** Partitions of each topic. */ - protected final Map topicPartitionsMap; - - /** Flag indicating whether to accept failures (and log them), or to fail on failures. */ - protected boolean logFailuresOnly; - - /** - * If true, the producer will wait until all outstanding records have been send to the broker. - */ - protected boolean flushOnCheckpoint = true; - - // -------------------------------- Runtime fields ------------------------------------------ - - /** KafkaProducer instance. */ - protected transient KafkaProducer producer; - - /** The callback than handles error propagation or logging callbacks. */ - protected transient Callback callback; - - /** Errors encountered in the async producer are stored here. */ - protected transient volatile Exception asyncException; - - /** Lock for accessing the pending records. */ - protected final SerializableObject pendingRecordsLock = new SerializableObject(); - - /** Number of unacknowledged records. */ - protected long pendingRecords; - - /** - * The main constructor for creating a FlinkKafkaProducer. - * - * @param defaultTopicId The default topic to write data to - * @param serializationSchema A serializable serialization schema for turning user objects into - * a kafka-consumable byte[] supporting key/value messages - * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is - * the only required argument. - * @param customPartitioner A serializable partitioner for assigning messages to Kafka - * partitions. Passing null will use Kafka's partitioner. - */ - public FlinkKafkaProducerBase( - String defaultTopicId, - KeyedSerializationSchema serializationSchema, - Properties producerConfig, - FlinkKafkaPartitioner customPartitioner) { - requireNonNull(defaultTopicId, "TopicID not set"); - requireNonNull(serializationSchema, "serializationSchema not set"); - requireNonNull(producerConfig, "producerConfig not set"); - ClosureCleaner.clean( - customPartitioner, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); - ClosureCleaner.ensureSerializable(serializationSchema); - - this.defaultTopicId = defaultTopicId; - this.schema = serializationSchema; - this.producerConfig = producerConfig; - this.flinkKafkaPartitioner = customPartitioner; - - // set the producer configuration properties for kafka record key value serializers. - if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getName()); - } else { - LOG.warn( - "Overwriting the '{}' is not recommended", - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); - } - - if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put( - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - ByteArraySerializer.class.getName()); - } else { - LOG.warn( - "Overwriting the '{}' is not recommended", - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); - } - - // eagerly ensure that bootstrap servers are set. - if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) { - throw new IllegalArgumentException( - ProducerConfig.BOOTSTRAP_SERVERS_CONFIG - + " must be supplied in the producer config properties."); - } - - this.topicPartitionsMap = new HashMap<>(); - } - - // ---------------------------------- Properties -------------------------- - - /** - * Defines whether the producer should fail on errors, or only log them. If this is set to true, - * then exceptions will be only logged, if set to false, exceptions will be eventually thrown - * and cause the streaming program to fail (and enter recovery). - * - * @param logFailuresOnly The flag to indicate logging-only on exceptions. - */ - public void setLogFailuresOnly(boolean logFailuresOnly) { - this.logFailuresOnly = logFailuresOnly; - } - - /** - * If set to true, the Flink producer will wait for all outstanding messages in the Kafka - * buffers to be acknowledged by the Kafka producer on a checkpoint. This way, the producer can - * guarantee that messages in the Kafka buffers are part of the checkpoint. - * - * @param flush Flag indicating the flushing mode (true = flush on checkpoint) - */ - public void setFlushOnCheckpoint(boolean flush) { - this.flushOnCheckpoint = flush; - } - - /** Used for testing only. */ - @VisibleForTesting - protected KafkaProducer getKafkaProducer(Properties props) { - return new KafkaProducer<>(props); - } - - // ----------------------------------- Utilities -------------------------- - - /** Initializes the connection to Kafka. */ - @Override - public void open(Configuration configuration) throws Exception { - if (schema instanceof KeyedSerializationSchemaWrapper) { - ((KeyedSerializationSchemaWrapper) schema) - .getSerializationSchema() - .open( - RuntimeContextInitializationContextAdapters.serializationAdapter( - getRuntimeContext(), - metricGroup -> metricGroup.addGroup("user"))); - } - producer = getKafkaProducer(this.producerConfig); - - RuntimeContext ctx = getRuntimeContext(); - - if (null != flinkKafkaPartitioner) { - flinkKafkaPartitioner.open( - ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); - } - - LOG.info( - "Starting FlinkKafkaProducer ({}/{}) to produce into default topic {}", - ctx.getIndexOfThisSubtask() + 1, - ctx.getNumberOfParallelSubtasks(), - defaultTopicId); - - // register Kafka metrics to Flink accumulators - if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { - Map metrics = this.producer.metrics(); - - if (metrics == null) { - // MapR's Kafka implementation returns null here. - LOG.info("Producer implementation does not support metrics"); - } else { - final MetricGroup kafkaMetricGroup = - getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); - for (Map.Entry metric : metrics.entrySet()) { - kafkaMetricGroup.gauge( - metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); - } - } - } - - if (flushOnCheckpoint - && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { - LOG.warn( - "Flushing on checkpoint is enabled, but checkpointing is not enabled. Disabling flushing."); - flushOnCheckpoint = false; - } - - if (logFailuresOnly) { - callback = - new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) { - LOG.error( - "Error while sending record to Kafka: " + e.getMessage(), - e); - } - acknowledgeMessage(); - } - }; - } else { - callback = - new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && asyncException == null) { - asyncException = exception; - } - acknowledgeMessage(); - } - }; - } - } - - /** - * Called when new data arrives to the sink, and forwards it to Kafka. - * - * @param next The incoming data - */ - @Override - public void invoke(IN next, Context context) throws Exception { - // propagate asynchronous errors - checkErroneous(); - - byte[] serializedKey = schema.serializeKey(next); - byte[] serializedValue = schema.serializeValue(next); - String targetTopic = schema.getTargetTopic(next); - if (targetTopic == null) { - targetTopic = defaultTopicId; - } - - int[] partitions = this.topicPartitionsMap.get(targetTopic); - if (null == partitions) { - partitions = getPartitionsByTopic(targetTopic, producer); - this.topicPartitionsMap.put(targetTopic, partitions); - } - - ProducerRecord record; - if (flinkKafkaPartitioner == null) { - record = new ProducerRecord<>(targetTopic, serializedKey, serializedValue); - } else { - record = - new ProducerRecord<>( - targetTopic, - flinkKafkaPartitioner.partition( - next, serializedKey, serializedValue, targetTopic, partitions), - serializedKey, - serializedValue); - } - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords++; - } - } - producer.send(record, callback); - } - - @Override - public void close() throws Exception { - if (producer != null) { - producer.close(); - } - - // make sure we propagate pending errors - checkErroneous(); - } - - // ------------------- Logic for handling checkpoint flushing -------------------------- // - - private void acknowledgeMessage() { - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords--; - if (pendingRecords == 0) { - pendingRecordsLock.notifyAll(); - } - } - } - } - - /** Flush pending records. */ - protected abstract void flush(); - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // nothing to do - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - // check for asynchronous errors and fail the checkpoint if necessary - checkErroneous(); - - if (flushOnCheckpoint) { - // flushing is activated: We need to wait until pendingRecords is 0 - flush(); - synchronized (pendingRecordsLock) { - if (pendingRecords != 0) { - throw new IllegalStateException( - "Pending record count must be zero at this point: " + pendingRecords); - } - - // if the flushed requests has errors, we should propagate it also and fail the - // checkpoint - checkErroneous(); - } - } - } - - // ----------------------------------- Utilities -------------------------- - - protected void checkErroneous() throws Exception { - Exception e = asyncException; - if (e != null) { - // prevent double throwing - asyncException = null; - throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e); - } - } - - public static Properties getPropertiesFromBrokerList(String brokerList) { - String[] elements = brokerList.split(","); - - // validate the broker addresses - for (String broker : elements) { - NetUtils.getCorrectHostnamePort(broker); - } - - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); - return props; - } - - protected static int[] getPartitionsByTopic( - String topic, KafkaProducer producer) { - // the fetched list is immutable, so we're creating a mutable copy in order to sort it - List partitionsList = new ArrayList<>(producer.partitionsFor(topic)); - - // sort the partitions by partition id to make sure the fetched partition list is the same - // across subtasks - Collections.sort( - partitionsList, - new Comparator() { - @Override - public int compare(PartitionInfo o1, PartitionInfo o2) { - return Integer.compare(o1.partition(), o2.partition()); - } - }); - - int[] partitions = new int[partitionsList.size()]; - for (int i = 0; i < partitions.length; i++) { - partitions[i] = partitionsList.get(i).partition(); - } - - return partitions; - } - - @VisibleForTesting - protected long numPendingRecords() { - synchronized (pendingRecordsLock) { - return pendingRecords; - } - } -} diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java index ad977cd95..d40139595 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaContextAware.java @@ -26,8 +26,11 @@ * *

      You only need to override the methods for the information that you need. However, {@link * #getTargetTopic(Object)} is required because it is used to determine the available partitions. + * + * @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaContextAware { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java index b54b9835f..8f15b921b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaDeserializationSchema.java @@ -31,8 +31,10 @@ * (Java/Scala objects) that are processed by Flink. * * @param The type created by the keyed deserialization schema. + * @deprecated Will be turned into internal API when {@link FlinkKafkaConsumer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaDeserializationSchema extends Serializable, ResultTypeQueryable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java index 89e2b9209..7ed987fce 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializationSchema.java @@ -35,8 +35,10 @@ * which the Kafka Producer is running. * * @param the type of values being serialized + * @deprecated Will be turned into internal API when {@link FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public interface KafkaSerializationSchema extends Serializable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java index 32b9d4cdc..6ad4f8337 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitMode.java @@ -26,6 +26,7 @@ *

      The exact value of this is determined at runtime in the consumer subtasks. */ @Internal +@Deprecated public enum OffsetCommitMode { /** Completely disable offset committing. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java index 1394af72f..32ac2f5f0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/config/OffsetCommitModes.java @@ -21,6 +21,7 @@ /** Utilities for {@link OffsetCommitMode}. */ @Internal +@Deprecated public class OffsetCommitModes { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java index 841d45288..074363021 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java @@ -61,6 +61,7 @@ * @param The type of topic/partition identifier used by Kafka in the specific version. */ @Internal +@Deprecated public abstract class AbstractFetcher { private static final int NO_TIMESTAMPS_WATERMARKS = 0; @@ -435,6 +436,14 @@ private List> createPartitionStateHolders( kafkaHandle, deserializedWatermarkStrategy.createTimestampAssigner( () -> consumerMetricGroup), + // When upgrading to Flink 2.0, context has to provide also + // the input activity clock. This is not trivial for the old + // sources. Ideally we should drop this old source before + // this connector is upgraded to Flink 2.0. Otherwise, we + // can avoid the compilation error without fixing the bug + // addressed by the FLIP-471, by returning SystemClock, + // which would reproduce the pre-FLIP-471 behavior (without + // fixing the underlying bug). deserializedWatermarkStrategy.createWatermarkGenerator( () -> consumerMetricGroup), immediateOutput, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 05e078f66..c8dc18360 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -19,8 +19,8 @@ import org.apache.flink.annotation.Internal; +import java.util.ArrayList; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Set; @@ -41,6 +41,7 @@ * allows the discoverer to be interrupted during a {@link #discoverPartitions()} call. */ @Internal +@Deprecated public abstract class AbstractPartitionDiscoverer { /** Describes whether we are discovering partitions for fixed topics or a topic pattern. */ @@ -130,21 +131,18 @@ public List discoverPartitions() throws WakeupException, Cl // topics or a topic pattern if (topicsDescriptor.isFixedTopics()) { newDiscoveredPartitions = - getAllPartitionsForTopics(topicsDescriptor.getFixedTopics()); + new ArrayList<>( + getAllPartitionsForTopics(topicsDescriptor.getFixedTopics())); } else { - List matchedTopics = getAllTopics(); + List matchedTopics = new ArrayList<>(getAllTopics()); // retain topics that match the pattern - Iterator iter = matchedTopics.iterator(); - while (iter.hasNext()) { - if (!topicsDescriptor.isMatchingTopic(iter.next())) { - iter.remove(); - } - } + matchedTopics.removeIf(s -> !topicsDescriptor.isMatchingTopic(s)); - if (matchedTopics.size() != 0) { + if (!matchedTopics.isEmpty()) { // get partitions only for matched topics - newDiscoveredPartitions = getAllPartitionsForTopics(matchedTopics); + newDiscoveredPartitions = + new ArrayList<>(getAllPartitionsForTopics(matchedTopics)); } else { newDiscoveredPartitions = null; } @@ -157,14 +155,8 @@ public List discoverPartitions() throws WakeupException, Cl "Unable to retrieve any partitions with KafkaTopicsDescriptor: " + topicsDescriptor); } else { - Iterator iter = newDiscoveredPartitions.iterator(); - KafkaTopicPartition nextPartition; - while (iter.hasNext()) { - nextPartition = iter.next(); - if (!setAndCheckDiscoveredPartition(nextPartition)) { - iter.remove(); - } - } + newDiscoveredPartitions.removeIf( + nextPartition -> !setAndCheckDiscoveredPartition(nextPartition)); } return newDiscoveredPartitions; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java index 22800611d..3b1751d40 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueue.java @@ -49,6 +49,7 @@ * @param The type of elements in the queue. */ @Internal +@Deprecated public class ClosableBlockingQueue { /** The lock used to make queue accesses and open checks atomic. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java index 204a01b4c..a9f9c9cae 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java @@ -65,6 +65,7 @@ * } */ @Internal +@Deprecated public class ExceptionProxy { /** The thread that should be interrupted when an exception occurs. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java index 4b270b0b7..6e618cbe0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/FlinkKafkaInternalProducer.java @@ -22,8 +22,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.Preconditions; -import org.apache.flink.shaded.guava30.com.google.common.base.Joiner; - import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.Callback; @@ -51,13 +49,16 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.concurrent.Future; +import java.util.stream.Collectors; /** Internal flink kafka producer. */ @PublicEvolving +@Deprecated public class FlinkKafkaInternalProducer implements Producer { private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaInternalProducer.class); @@ -169,7 +170,9 @@ public void close(Duration duration) { LOG.debug( "Closed internal KafkaProducer {}. Stacktrace: {}", System.identityHashCode(this), - Joiner.on("\n").join(Thread.currentThread().getStackTrace())); + Arrays.stream(Thread.currentThread().getStackTrace()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n"))); } closed = true; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java index 7fc50e625..64132b0b9 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Handover.java @@ -47,6 +47,7 @@ */ @ThreadSafe @Internal +@Deprecated public final class Handover implements Closeable { private final Object lock = new Object(); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java index d7666772b..f1180b8b5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaCommitCallback.java @@ -25,6 +25,7 @@ * commit request completes, which should normally be triggered from checkpoint complete event. */ @Internal +@Deprecated public interface KafkaCommitCallback { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java index f7f40b80b..5b6fb4d43 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaConsumerThread.java @@ -61,6 +61,7 @@ * an indirection to the KafkaConsumer calls that change signature. */ @Internal +@Deprecated public class KafkaConsumerThread extends Thread { /** Logger for this consumer. */ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java index d53e4ff4d..b754b4d09 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaDeserializationSchemaWrapper.java @@ -32,6 +32,7 @@ * @param The type created by the deserialization schema. */ @Internal +@Deprecated public class KafkaDeserializationSchemaWrapper implements KafkaDeserializationSchema { private static final long serialVersionUID = 2651665280744549932L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java index bee233975..428e6c7ce 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaFetcher.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedValue; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -50,6 +51,7 @@ * @param The type of elements produced by the fetcher. */ @Internal +@Deprecated public class KafkaFetcher extends AbstractFetcher { private static final Logger LOG = LoggerFactory.getLogger(KafkaFetcher.class); @@ -142,6 +144,12 @@ public void runFetchLoop() throws Exception { partitionConsumerRecordsHandler(partitionRecords, partition); } } + } catch (Handover.ClosedException ex) { + if (running) { + // rethrow, only if we are running, if fetcher is not running we should not throw + // the ClosedException, as we are stopping gracefully + ExceptionUtils.rethrowException(ex); + } } finally { // this signals the consumer thread that no more work is to be done consumerThread.shutdown(); diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java index ec788991d..ef7162bde 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionDiscoverer.java @@ -34,6 +34,7 @@ * brokers via the Kafka high-level consumer API. */ @Internal +@Deprecated public class KafkaPartitionDiscoverer extends AbstractPartitionDiscoverer { private final Properties kafkaProperties; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java index 73b1d42a6..147fad9b6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaSerializationSchemaWrapper.java @@ -35,6 +35,7 @@ * KafkaSerializationSchema}. */ @Internal +@Deprecated public class KafkaSerializationSchemaWrapper implements KafkaSerializationSchema, KafkaContextAware { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java index fe7ee7f7b..c61db83f0 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaShuffleFetcher.java @@ -50,6 +50,7 @@ /** Fetch data from Kafka for Kafka Shuffle. */ @Internal +@Deprecated public class KafkaShuffleFetcher extends KafkaFetcher { /** The handler to check and generate watermarks from fetched records. * */ private final WatermarkHandler watermarkHandler; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index f262a222c..0e91042f6 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -32,8 +32,13 @@ * *

      Note: This class must not change in its structure, because it would change the serialization * format and make previous savepoints unreadable. + * + * @deprecated Will be turned into internal class when {@link + * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. Replace with + * {@link org.apache.kafka.common.TopicPartition}. */ @PublicEvolving +@Deprecated public final class KafkaTopicPartition implements Serializable { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java index be61e8ad7..83c7483ff 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionAssigner.java @@ -21,6 +21,7 @@ /** Utility for assigning Kafka partitions to consumer subtasks. */ @Internal +@Deprecated public class KafkaTopicPartitionAssigner { /** diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java index a2ef12882..031400d6e 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionLeader.java @@ -27,6 +27,7 @@ * Serializable Topic Partition info with leader Node information. This class is used at runtime. */ @Internal +@Deprecated public class KafkaTopicPartitionLeader implements Serializable { private static final long serialVersionUID = 9145855900303748582L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java index c09df342c..ee669e7e1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java @@ -29,6 +29,7 @@ * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ @Internal +@Deprecated public class KafkaTopicPartitionState { // ------------------------------------------------------------------------ diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java index 6c843409d..f9c815fcc 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithWatermarkGenerator.java @@ -34,6 +34,7 @@ * @param The type of the Kafka partition descriptor, which varies across Kafka versions. */ @Internal +@Deprecated public final class KafkaTopicPartitionStateWithWatermarkGenerator extends KafkaTopicPartitionState { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java index 8261a2b31..4bb37b1c2 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptor.java @@ -33,6 +33,7 @@ * list of topics, or a topic pattern. */ @Internal +@Deprecated public class KafkaTopicsDescriptor implements Serializable { private static final long serialVersionUID = -3807227764764900975L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java index c95cd9c40..ae4e922d5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KeyedSerializationSchemaWrapper.java @@ -27,6 +27,7 @@ * @param The type to serialize */ @Internal +@Deprecated public class KeyedSerializationSchemaWrapper implements KeyedSerializationSchema { private static final long serialVersionUID = 1351665280744549933L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java index 8e5674313..68c4db12a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SourceContextWatermarkOutputAdapter.java @@ -25,6 +25,7 @@ * A {@link org.apache.flink.api.common.eventtime.WatermarkOutput} that forwards calls to a {@link * org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. */ +@Deprecated public class SourceContextWatermarkOutputAdapter implements WatermarkOutput { private final SourceContext sourceContext; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java index e21355e8f..cd6270acc 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TransactionalIdsGenerator.java @@ -40,6 +40,7 @@ * subtask. */ @Internal +@Deprecated public class TransactionalIdsGenerator { private final String prefix; private final int subtaskIndex; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java index 30d469714..731089028 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaConsumerMetricConstants.java @@ -26,6 +26,7 @@ * metrics. */ @Internal +@Deprecated public class KafkaConsumerMetricConstants { public static final String KAFKA_CONSUMER_METRICS_GROUP = "KafkaConsumer"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java index 1ab41ce9c..2893bad33 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricWrapper.java @@ -23,6 +23,7 @@ /** Gauge for getting the current value of a Kafka metric. */ @Internal +@Deprecated public class KafkaMetricWrapper implements Gauge { private final org.apache.kafka.common.Metric kafkaMetric; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java index 16b57f621..e70baea3b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkFixedPartitioner.java @@ -54,8 +54,12 @@ *

      Not all Kafka partitions contain data To avoid such an unbalanced partitioning, use a * round-robin kafka partitioner (note that this will cause a lot of network connections between all * the Flink instances and all the Kafka brokers). + * + * @deprecated Will be turned into internal class when {@link + * org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer} is removed. */ @PublicEvolving +@Deprecated public class FlinkFixedPartitioner extends FlinkKafkaPartitioner { private static final long serialVersionUID = -3785320239953858777L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java index 2fb89e205..9568349a3 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FlinkKafkaPartitioner.java @@ -18,39 +18,18 @@ package org.apache.flink.streaming.connectors.kafka.partitioner; import org.apache.flink.annotation.PublicEvolving; - -import java.io.Serializable; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; /** * A {@link FlinkKafkaPartitioner} wraps logic on how to partition records across partitions of * multiple Kafka topics. + * + * @deprecated Use {@link KafkaPartitioner} instead for {@link + * org.apache.flink.connector.kafka.sink.KafkaSink}. */ @PublicEvolving -public abstract class FlinkKafkaPartitioner implements Serializable { +@Deprecated +public abstract class FlinkKafkaPartitioner implements KafkaPartitioner { private static final long serialVersionUID = -9086719227828020494L; - - /** - * Initializer for the partitioner. This is called once on each parallel sink instance of the - * Flink Kafka producer. This method should be overridden if necessary. - * - * @param parallelInstanceId 0-indexed id of the parallel sink instance in Flink - * @param parallelInstances the total number of parallel instances - */ - public void open(int parallelInstanceId, int parallelInstances) { - // overwrite this method if needed. - } - - /** - * Determine the id of the partition that the record should be written to. - * - * @param record the record value - * @param key serialized key of the record - * @param value serialized value of the record - * @param targetTopic target topic for the record - * @param partitions found partitions for the target topic - * @return the id of the target partition - */ - public abstract int partition( - T record, byte[] key, byte[] value, String targetTopic, int[] partitions); } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java index ae9af29f0..bb7c76a67 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffle.java @@ -98,8 +98,13 @@ * | * | ----------> KafkaShuffleConsumerReuse -> ... * + * + * @deprecated This experimental feature never graduated to a stable feature and will be removed in + * future releases. In case of interest to port it to the Source/Sink API, please reach out to + * the Flink community. */ @Experimental +@Deprecated public class FlinkKafkaShuffle { static final String PRODUCER_PARALLELISM = "producer parallelism"; static final String PARTITION_NUMBER = "partition number"; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java index 886343bea..b96e9c0f5 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleConsumer.java @@ -39,6 +39,7 @@ /** Flink Kafka Shuffle Consumer Function. */ @Internal +@Deprecated public class FlinkKafkaShuffleConsumer extends FlinkKafkaConsumer { private final TypeSerializer typeSerializer; private final int producerParallelism; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java index e05e8f9a3..46754f270 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/FlinkKafkaShuffleProducer.java @@ -44,6 +44,7 @@ * handling elements and watermarks */ @Internal +@Deprecated public class FlinkKafkaShuffleProducer extends FlinkKafkaProducer { private final KafkaSerializer kafkaSerializer; private final KeySelector keySelector; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java index 8bd77840f..e24e15650 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/shuffle/StreamKafkaShuffleSink.java @@ -29,6 +29,7 @@ * this way to avoid public interface change. */ @Internal +@Deprecated class StreamKafkaShuffleSink extends StreamSink { public StreamKafkaShuffleSink(FlinkKafkaShuffleProducer flinkKafkaShuffleProducer) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java index 91798281d..ef5eca95b 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaDeserializationSchema.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; @@ -38,6 +39,7 @@ import java.util.List; /** A specific {@link KafkaSerializationSchema} for {@link KafkaDynamicSource}. */ +@Internal class DynamicKafkaDeserializationSchema implements KafkaDeserializationSchema { private static final long serialVersionUID = 1L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java index 7908aded6..229b08b5f 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -17,10 +17,11 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; import org.apache.flink.connector.kafka.sink.KafkaSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; @@ -30,13 +31,22 @@ import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + import static org.apache.flink.util.Preconditions.checkNotNull; /** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. */ +@Internal class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { - private final String topic; - private final FlinkKafkaPartitioner partitioner; + private final Set topics; + private final Pattern topicPattern; + private final KafkaPartitioner partitioner; @Nullable private final SerializationSchema keySerialization; private final SerializationSchema valueSerialization; private final RowData.FieldGetter[] keyFieldGetters; @@ -44,10 +54,12 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS private final boolean hasMetadata; private final int[] metadataPositions; private final boolean upsertMode; + private final Map topicPatternMatches; DynamicKafkaRecordSerializationSchema( - String topic, - @Nullable FlinkKafkaPartitioner partitioner, + @Nullable List topics, + @Nullable Pattern topicPattern, + @Nullable KafkaPartitioner partitioner, @Nullable SerializationSchema keySerialization, SerializationSchema valueSerialization, RowData.FieldGetter[] keyFieldGetters, @@ -60,7 +72,16 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS keySerialization != null && keyFieldGetters.length > 0, "Key must be set in upsert mode for serialization schema."); } - this.topic = checkNotNull(topic); + Preconditions.checkArgument( + (topics != null && topicPattern == null && topics.size() > 0) + || (topics == null && topicPattern != null), + "Either Topic or Topic Pattern must be set."); + if (topics != null) { + this.topics = new HashSet<>(topics); + } else { + this.topics = null; + } + this.topicPattern = topicPattern; this.partitioner = partitioner; this.keySerialization = keySerialization; this.valueSerialization = checkNotNull(valueSerialization); @@ -69,6 +90,8 @@ class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationS this.hasMetadata = hasMetadata; this.metadataPositions = metadataPositions; this.upsertMode = upsertMode; + // Cache results of topic pattern matches to avoid re-evaluating the pattern for each record + this.topicPatternMatches = new HashMap<>(); } @Override @@ -77,13 +100,15 @@ public ProducerRecord serialize( // shortcut in case no input projection is required if (keySerialization == null && !hasMetadata) { final byte[] valueSerialized = valueSerialization.serialize(consumedRow); + final String targetTopic = getTargetTopic(consumedRow); return new ProducerRecord<>( - topic, + targetTopic, extractPartition( consumedRow, + targetTopic, null, valueSerialized, - context.getPartitionsForTopic(topic)), + context.getPartitionsForTopic(targetTopic)), null, valueSerialized); } @@ -115,14 +140,15 @@ public ProducerRecord serialize( consumedRow, kind, valueFieldGetters); valueSerialized = valueSerialization.serialize(valueRow); } - + final String targetTopic = getTargetTopic(consumedRow); return new ProducerRecord<>( - topic, + targetTopic, extractPartition( consumedRow, + targetTopic, keySerialized, valueSerialized, - context.getPartitionsForTopic(topic)), + context.getPartitionsForTopic(targetTopic)), readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.TIMESTAMP), keySerialized, valueSerialized, @@ -144,14 +170,42 @@ public void open( valueSerialization.open(context); } + private String getTargetTopic(RowData element) { + if (topics != null && topics.size() == 1) { + // If topics is a singleton list, we only return the provided topic. + return topics.stream().findFirst().get(); + } + final String targetTopic = readMetadata(element, KafkaDynamicSink.WritableMetadata.TOPIC); + if (targetTopic == null) { + throw new IllegalArgumentException( + "The topic of the sink record is not valid. Expected a single topic but no topic is set."); + } else if (topics != null && !topics.contains(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to be in: %s but was: %s", + topics, targetTopic)); + } else if (topicPattern != null && !cachedTopicPatternMatch(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to match: %s but was: %s", + topicPattern, targetTopic)); + } + return targetTopic; + } + + private boolean cachedTopicPatternMatch(String topic) { + return topicPatternMatches.computeIfAbsent(topic, t -> topicPattern.matcher(t).matches()); + } + private Integer extractPartition( RowData consumedRow, + String targetTopic, @Nullable byte[] keySerialized, byte[] valueSerialized, int[] partitions) { if (partitioner != null) { return partitioner.partition( - consumedRow, keySerialized, valueSerialized, topic, partitions); + consumedRow, keySerialized, valueSerialized, targetTopic, partitions); } return null; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java index a6cdbcedc..c64ab0bef 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java @@ -106,6 +106,7 @@ public class KafkaConnectorOptions { ValueFieldsStrategy.EXCEPT_KEY)) .build()); + public static final ConfigOption SCAN_PARALLELISM = FactoryUtil.SOURCE_PARALLELISM; public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; // -------------------------------------------------------------------------------------------- @@ -118,15 +119,15 @@ public class KafkaConnectorOptions { .asList() .noDefaultValue() .withDescription( - "Topic names from which the table is read. Either 'topic' or 'topic-pattern' must be set for source. " - + "Option 'topic' is required for sink."); + "Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by semicolon like 'topic-1;topic-2'. Note, only one of 'topic-pattern' and 'topic' can be specified for sources. " + + "When the table is used as sink, the topic name is the topic to write data. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified."); public static final ConfigOption TOPIC_PATTERN = ConfigOptions.key("topic-pattern") .stringType() .noDefaultValue() .withDescription( - "Optional topic pattern from which the table is read for source. Either 'topic' or 'topic-pattern' must be set."); + "Optional topic pattern from which the table is read for source, or topic pattern that must match the provided `topic` metadata column for sink. Either 'topic' or 'topic-pattern' must be set."); public static final ConfigOption PROPS_BOOTSTRAP_SERVERS = ConfigOptions.key("properties.bootstrap.servers") @@ -188,9 +189,11 @@ public class KafkaConnectorOptions { public static final ConfigOption SCAN_TOPIC_PARTITION_DISCOVERY = ConfigOptions.key("scan.topic-partition-discovery.interval") .durationType() - .noDefaultValue() + .defaultValue(Duration.ofMinutes(5)) .withDescription( - "Optional interval for consumer to discover dynamically created Kafka partitions periodically."); + "Optional interval for consumer to discover dynamically created Kafka partitions periodically." + + "The value 0 disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); // -------------------------------------------------------------------------------------------- // Sink specific options diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java index ef70644e5..5960a709a 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java @@ -24,11 +24,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ValueFieldsStrategy; @@ -98,23 +98,22 @@ class KafkaConnectorOptionsUtil { protected static final String DEBEZIUM_AVRO_CONFLUENT = "debezium-avro-confluent"; private static final List SCHEMA_REGISTRY_FORMATS = Arrays.asList(AVRO_CONFLUENT, DEBEZIUM_AVRO_CONFLUENT); - // -------------------------------------------------------------------------------------------- // Validation // -------------------------------------------------------------------------------------------- public static void validateTableSourceOptions(ReadableConfig tableOptions) { - validateSourceTopic(tableOptions); + validateTopic(tableOptions); validateScanStartupMode(tableOptions); validateScanBoundedMode(tableOptions); } public static void validateTableSinkOptions(ReadableConfig tableOptions) { - validateSinkTopic(tableOptions); + validateTopic(tableOptions); validateSinkPartitioner(tableOptions); } - public static void validateSourceTopic(ReadableConfig tableOptions) { + public static void validateTopic(ReadableConfig tableOptions) { Optional> topic = tableOptions.getOptional(TOPIC); Optional pattern = tableOptions.getOptional(TOPIC_PATTERN); @@ -128,23 +127,6 @@ public static void validateSourceTopic(ReadableConfig tableOptions) { } } - public static void validateSinkTopic(ReadableConfig tableOptions) { - String errorMessageTemp = - "Flink Kafka sink currently only supports single topic, but got %s: %s."; - if (!isSingleTopic(tableOptions)) { - if (tableOptions.getOptional(TOPIC_PATTERN).isPresent()) { - throw new ValidationException( - String.format( - errorMessageTemp, - "'topic-pattern'", - tableOptions.get(TOPIC_PATTERN))); - } else { - throw new ValidationException( - String.format(errorMessageTemp, "'topic'", tableOptions.get(TOPIC))); - } - } - } - private static void validateScanStartupMode(ReadableConfig tableOptions) { tableOptions .getOptional(SCAN_STARTUP_MODE) @@ -189,7 +171,7 @@ private static void validateScanStartupMode(ReadableConfig tableOptions) { }); } - private static void validateScanBoundedMode(ReadableConfig tableOptions) { + static void validateScanBoundedMode(ReadableConfig tableOptions) { tableOptions .getOptional(SCAN_BOUNDED_MODE) .ifPresent( @@ -254,11 +236,11 @@ private static void validateSinkPartitioner(ReadableConfig tableOptions) { // Utilities // -------------------------------------------------------------------------------------------- - public static List getSourceTopics(ReadableConfig tableOptions) { + public static List getTopics(ReadableConfig tableOptions) { return tableOptions.getOptional(TOPIC).orElse(null); } - public static Pattern getSourceTopicPattern(ReadableConfig tableOptions) { + public static Pattern getTopicPattern(ReadableConfig tableOptions) { return tableOptions.getOptional(TOPIC_PATTERN).map(Pattern::compile).orElse(null); } @@ -404,7 +386,7 @@ public static Properties getKafkaProperties(Map tableOptions) { * The partitioner can be either "fixed", "round-robin" or a customized partitioner full class * name. */ - public static Optional> getFlinkKafkaPartitioner( + public static Optional> getFlinkKafkaPartitioner( ReadableConfig tableOptions, ClassLoader classLoader) { return tableOptions .getOptional(SINK_PARTITIONER) @@ -483,19 +465,19 @@ private static boolean hasKafkaClientProperties(Map tableOptions } /** Returns a class value with the given class name. */ - private static FlinkKafkaPartitioner initializePartitioner( + private static KafkaPartitioner initializePartitioner( String name, ClassLoader classLoader) { try { Class clazz = Class.forName(name, true, classLoader); - if (!FlinkKafkaPartitioner.class.isAssignableFrom(clazz)) { + if (!KafkaPartitioner.class.isAssignableFrom(clazz)) { throw new ValidationException( String.format( - "Sink partitioner class '%s' should extend from the required class %s", - name, FlinkKafkaPartitioner.class.getName())); + "Sink partitioner class '%s' should implement the required class %s", + name, KafkaPartitioner.class.getName())); } @SuppressWarnings("unchecked") - final FlinkKafkaPartitioner kafkaPartitioner = - InstantiationUtil.instantiate(name, FlinkKafkaPartitioner.class, classLoader); + final KafkaPartitioner kafkaPartitioner = + InstantiationUtil.instantiate(name, KafkaPartitioner.class, classLoader); return kafkaPartitioner; } catch (ClassNotFoundException | FlinkException e) { @@ -636,21 +618,25 @@ public static DynamicTableFactory.Context autoCompleteSchemaRegistrySubject( private static Map autoCompleteSchemaRegistrySubject( Map options) { Configuration configuration = Configuration.fromMap(options); - // the subject autoComplete should only be used in sink, check the topic first - validateSinkTopic(configuration); - final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); - final Optional keyFormat = configuration.getOptional(KEY_FORMAT); - final Optional format = configuration.getOptional(FORMAT); - final String topic = configuration.get(TOPIC).get(0); - - if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { - autoCompleteSubject(configuration, format.get(), topic + "-value"); - } else if (valueFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { - autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); - } + // the subject autoComplete should only be used in sink with a single topic, check the topic + // option first + validateTopic(configuration); + if (configuration.contains(TOPIC) && isSingleTopic(configuration)) { + final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); + final Optional keyFormat = configuration.getOptional(KEY_FORMAT); + final Optional format = configuration.getOptional(FORMAT); + final String topic = configuration.get(TOPIC).get(0); + + if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { + autoCompleteSubject(configuration, format.get(), topic + "-value"); + } else if (valueFormat.isPresent() + && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { + autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); + } - if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { - autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { + autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + } } return configuration.toMap(); } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java index 041846dd7..2bb52c94d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.Projection; @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.regex.Pattern; import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -114,14 +115,17 @@ public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetada */ @Nullable private final String transactionalIdPrefix; - /** The Kafka topic to write to. */ - protected final String topic; + /** The Kafka topics to allow for producing. */ + protected final List topics; + + /** The Kafka topic pattern of topics allowed to produce to. */ + protected final Pattern topicPattern; /** Properties for the Kafka producer. */ protected final Properties properties; /** Partitioner to select Kafka partition for each item. */ - protected final @Nullable FlinkKafkaPartitioner partitioner; + protected final @Nullable KafkaPartitioner partitioner; /** * Flag to determine sink mode. In upsert mode sink transforms the delete/update-before message @@ -143,9 +147,10 @@ public KafkaDynamicSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, - @Nullable FlinkKafkaPartitioner partitioner, + @Nullable KafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, boolean upsertMode, SinkBufferFlushMode flushMode, @@ -166,7 +171,8 @@ public KafkaDynamicSink( // Mutable attributes this.metadataKeys = Collections.emptyList(); // Kafka-specific attributes - this.topic = checkNotNull(topic, "Topic must not be null."); + this.topics = topics; + this.topicPattern = topicPattern; this.properties = checkNotNull(properties, "Properties must not be null."); this.partitioner = partitioner; this.deliveryGuarantee = @@ -206,7 +212,8 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { .setKafkaProducerConfig(properties) .setRecordSerializer( new DynamicKafkaRecordSerializationSchema( - topic, + topics, + topicPattern, partitioner, keySerialization, valueSerialization, @@ -223,7 +230,7 @@ public DataStreamSink consumeDataStream( ProviderContext providerContext, DataStream dataStream) { final boolean objectReuse = dataStream.getExecutionEnvironment().getConfig().isObjectReuseEnabled(); - final ReducingUpsertSink sink = + final ReducingUpsertSink sink = new ReducingUpsertSink<>( kafkaSink, physicalDataType, @@ -250,8 +257,13 @@ public DataStreamSink consumeDataStream( @Override public Map listWritableMetadata() { final Map metadataMap = new LinkedHashMap<>(); - Stream.of(WritableMetadata.values()) - .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + for (WritableMetadata m : WritableMetadata.values()) { + if (topics != null && topics.size() == 1 && WritableMetadata.TOPIC.key.equals(m.key)) { + // When `topic` is a singleton list, TOPIC metadata is not writable + continue; + } + metadataMap.put(m.key, m.dataType); + } return metadataMap; } @@ -272,7 +284,8 @@ public DynamicTableSink copy() { keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, @@ -306,7 +319,8 @@ public boolean equals(Object o) { && Arrays.equals(keyProjection, that.keyProjection) && Arrays.equals(valueProjection, that.valueProjection) && Objects.equals(keyPrefix, that.keyPrefix) - && Objects.equals(topic, that.topic) + && Objects.equals(topics, that.topics) + && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern)) && Objects.equals(properties, that.properties) && Objects.equals(partitioner, that.partitioner) && Objects.equals(deliveryGuarantee, that.deliveryGuarantee) @@ -327,7 +341,8 @@ public int hashCode() { keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, @@ -393,6 +408,20 @@ private RowData.FieldGetter[] getFieldGetters( // -------------------------------------------------------------------------------------------- enum WritableMetadata { + TOPIC( + "topic", + DataTypes.STRING().notNull(), + new MetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return row.getString(pos).toString(); + } + }), HEADERS( "headers", // key and value of the map are nullable to make handling easier in queries diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java index c963da762..012068085 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java @@ -71,6 +71,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Properties; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -171,6 +172,9 @@ public class KafkaDynamicSource protected final String tableIdentifier; + /** Parallelism of the physical Kafka consumer. * */ + protected final @Nullable Integer parallelism; + public KafkaDynamicSource( DataType physicalDataType, @Nullable DecodingFormat> keyDecodingFormat, @@ -188,7 +192,8 @@ public KafkaDynamicSource( Map specificBoundedOffsets, long boundedTimestampMillis, boolean upsertMode, - String tableIdentifier) { + String tableIdentifier, + @Nullable Integer parallelism) { // Format attributes this.physicalDataType = Preconditions.checkNotNull( @@ -228,6 +233,7 @@ public KafkaDynamicSource( this.boundedTimestampMillis = boundedTimestampMillis; this.upsertMode = upsertMode; this.tableIdentifier = tableIdentifier; + this.parallelism = parallelism; } @Override @@ -267,6 +273,11 @@ public DataStream produceDataStream( public boolean isBounded() { return kafkaSource.getBoundedness() == Boundedness.BOUNDED; } + + @Override + public Optional getParallelism() { + return Optional.ofNullable(parallelism); + } }; } @@ -344,7 +355,8 @@ public DynamicTableSource copy() { specificBoundedOffsets, boundedTimestampMillis, upsertMode, - tableIdentifier); + tableIdentifier, + parallelism); copy.producedDataType = producedDataType; copy.metadataKeys = metadataKeys; copy.watermarkStrategy = watermarkStrategy; @@ -384,7 +396,8 @@ public boolean equals(Object o) { && boundedTimestampMillis == that.boundedTimestampMillis && Objects.equals(upsertMode, that.upsertMode) && Objects.equals(tableIdentifier, that.tableIdentifier) - && Objects.equals(watermarkStrategy, that.watermarkStrategy); + && Objects.equals(watermarkStrategy, that.watermarkStrategy) + && Objects.equals(parallelism, that.parallelism); } @Override @@ -409,7 +422,8 @@ public int hashCode() { boundedTimestampMillis, upsertMode, tableIdentifier, - watermarkStrategy); + watermarkStrategy, + parallelism); } // -------------------------------------------------------------------------------------------- diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java index 48c00918a..34f57ff15 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactory.java @@ -26,11 +26,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; import org.apache.flink.connector.kafka.source.KafkaSourceOptions; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ObjectIdentifier; @@ -74,6 +74,7 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; @@ -93,9 +94,9 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getFlinkKafkaPartitioner; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getStartupOptions; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopics; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSinkOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateTableSourceOptions; @@ -152,6 +153,7 @@ public Set> optionalOptions() { options.add(SCAN_BOUNDED_MODE); options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(SCAN_PARALLELISM); return options; } @@ -166,6 +168,7 @@ public Set> forwardOptions() { SCAN_STARTUP_SPECIFIC_OFFSETS, SCAN_TOPIC_PARTITION_DISCOVERY, SCAN_STARTUP_TIMESTAMP_MILLIS, + SCAN_PARALLELISM, SINK_PARTITIONER, SINK_PARALLELISM, TRANSACTIONAL_ID_PREFIX) @@ -201,11 +204,11 @@ public DynamicTableSource createDynamicTableSource(Context context) { final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); // add topic-partition discovery - final Optional partitionDiscoveryInterval = - tableOptions.getOptional(SCAN_TOPIC_PARTITION_DISCOVERY).map(Duration::toMillis); + final Duration partitionDiscoveryInterval = + tableOptions.get(SCAN_TOPIC_PARTITION_DISCOVERY); properties.setProperty( KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - partitionDiscoveryInterval.orElse(-1L).toString()); + Long.toString(partitionDiscoveryInterval.toMillis())); final DataType physicalDataType = context.getPhysicalRowDataType(); @@ -215,6 +218,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + final Integer parallelism = tableOptions.getOptional(SCAN_PARALLELISM).orElse(null); + return createKafkaTableSource( physicalDataType, keyDecodingFormat.orElse(null), @@ -222,8 +227,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { keyProjection, valueProjection, keyPrefix, - getSourceTopics(tableOptions), - getSourceTopicPattern(tableOptions), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, startupOptions.startupMode, startupOptions.specificOffsets, @@ -231,7 +236,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { boundedOptions.boundedMode, boundedOptions.specificOffsets, boundedOptions.boundedTimestampMillis, - context.getObjectIdentifier().asSummaryString()); + context.getObjectIdentifier().asSummaryString(), + parallelism); } @Override @@ -278,7 +284,8 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyProjection, valueProjection, keyPrefix, - tableOptions.get(TOPIC).get(0), + getTopics(tableOptions), + getTopicPattern(tableOptions), getKafkaProperties(context.getCatalogTable().getOptions()), getFlinkKafkaPartitioner(tableOptions, context.getClassLoader()).orElse(null), deliveryGuarantee, @@ -395,7 +402,8 @@ protected KafkaDynamicSource createKafkaTableSource( BoundedMode boundedMode, Map specificEndOffsets, long endTimestampMillis, - String tableIdentifier) { + String tableIdentifier, + Integer parallelism) { return new KafkaDynamicSource( physicalDataType, keyDecodingFormat, @@ -413,7 +421,8 @@ protected KafkaDynamicSource createKafkaTableSource( specificEndOffsets, endTimestampMillis, false, - tableIdentifier); + tableIdentifier, + parallelism); } protected KafkaDynamicSink createKafkaTableSink( @@ -423,9 +432,10 @@ protected KafkaDynamicSink createKafkaTableSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, - FlinkKafkaPartitioner partitioner, + KafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, Integer parallelism, @Nullable String transactionalIdPrefix) { @@ -437,7 +447,8 @@ protected KafkaDynamicSink createKafkaTableSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java index da48e39e4..7cbe5f6cb 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java @@ -19,7 +19,9 @@ import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.api.connector.sink.SinkWriter; -import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; @@ -32,19 +34,21 @@ * A wrapper of a {@link Sink}. It will buffer the data emitted by the wrapper {@link SinkWriter} * and only emit it when the buffer is full or a timer is triggered or a checkpoint happens. * - *

      The sink provides eventual consistency guarantees without the need of a two-phase protocol - * because the updates are idempotent therefore duplicates have no effect. + *

      The sink provides eventual consistency guarantees under {@link + * org.apache.flink.connector.base.DeliveryGuarantee#AT_LEAST_ONCE} because the updates are + * idempotent therefore duplicates have no effect. */ -class ReducingUpsertSink implements StatefulSink { +class ReducingUpsertSink + implements TwoPhaseCommittingStatefulSink { - private final StatefulSink wrappedSink; + private final TwoPhaseCommittingStatefulSink wrappedSink; private final DataType physicalDataType; private final int[] keyProjection; private final SinkBufferFlushMode bufferFlushMode; private final SerializableFunction valueCopyFunction; ReducingUpsertSink( - StatefulSink wrappedSink, + TwoPhaseCommittingStatefulSink wrappedSink, DataType physicalDataType, int[] keyProjection, SinkBufferFlushMode bufferFlushMode, @@ -57,12 +61,11 @@ class ReducingUpsertSink implements StatefulSink createWriter(InitContext context) - throws IOException { - final StatefulSinkWriter wrapperWriter = - wrappedSink.createWriter(context); + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + createWriter(InitContext context) throws IOException { return new ReducingUpsertWriter<>( - wrapperWriter, + wrappedSink.createWriter(context), physicalDataType, keyProjection, bufferFlushMode, @@ -71,12 +74,25 @@ public StatefulSinkWriter createWriter(InitContext context } @Override - public StatefulSinkWriter restoreWriter( - InitContext context, Collection recoveredState) throws IOException { - final StatefulSinkWriter wrapperWriter = - wrappedSink.restoreWriter(context, recoveredState); + public Committer createCommitter(CommitterInitContext context) throws IOException { + return wrappedSink.createCommitter(context); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return wrappedSink.getCommittableSerializer(); + } + + @Override + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + restoreWriter(InitContext context, Collection recoveredState) + throws IOException { + final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter = wrappedSink.restoreWriter(context, recoveredState); return new ReducingUpsertWriter<>( - wrapperWriter, + wrappedWriter, physicalDataType, keyProjection, bufferFlushMode, diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java index 5a87add74..45421df64 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriter.java @@ -19,8 +19,8 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.connector.sink2.StatefulSink; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,10 +40,13 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -class ReducingUpsertWriter - implements StatefulSink.StatefulSinkWriter { +class ReducingUpsertWriter + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> { - private final StatefulSink.StatefulSinkWriter wrappedWriter; + private final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter; private final WrappedContext wrappedContext = new WrappedContext(); private final int batchMaxRowNums; private final Function valueCopyFunction; @@ -55,7 +59,9 @@ class ReducingUpsertWriter private long lastFlush = System.currentTimeMillis(); ReducingUpsertWriter( - StatefulSink.StatefulSinkWriter wrappedWriter, + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, WriterState, Comm> + wrappedWriter, DataType physicalDataType, int[] keyProjection, SinkBufferFlushMode bufferFlushMode, @@ -87,7 +93,8 @@ public void write(RowData element, Context context) throws IOException, Interrup @Override public void flush(boolean endOfInput) throws IOException, InterruptedException { - flush(); + sinkBuffer(); + wrappedWriter.flush(endOfInput); } @Override @@ -109,7 +116,7 @@ private void addToBuffer(RowData row, Long timestamp) throws IOException, Interr reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp)); if (reduceBuffer.size() >= batchMaxRowNums) { - flush(); + sinkBuffer(); } } @@ -121,7 +128,7 @@ private void registerFlush() { lastFlush + batchIntervalMs, (t) -> { if (t >= lastFlush + batchIntervalMs) { - flush(); + sinkBuffer(); } registerFlush(); }); @@ -140,7 +147,7 @@ private RowData changeFlag(RowData value) { return value; } - private void flush() throws IOException, InterruptedException { + private void sinkBuffer() throws IOException, InterruptedException { for (Tuple2 value : reduceBuffer.values()) { wrappedContext.setTimestamp(value.f1); wrappedWriter.write(value.f0, wrappedContext); @@ -149,6 +156,11 @@ private void flush() throws IOException, InterruptedException { reduceBuffer.clear(); } + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return wrappedWriter.prepareCommit(); + } + /** * Wrapper of {@link SinkWriter.Context}. * @@ -157,7 +169,7 @@ private void flush() throws IOException, InterruptedException { * ReducingUpsertWriter} will emit the records in the buffer with memorized timestamp. */ private static class WrappedContext implements SinkWriter.Context { - private long timestamp; + private Long timestamp; private SinkWriter.Context context; @Override @@ -168,11 +180,10 @@ public long currentWatermark() { @Override public Long timestamp() { - checkNotNull(timestamp, "timestamp must to be set before retrieving it."); return timestamp; } - public void setTimestamp(long timestamp) { + public void setTimestamp(Long timestamp) { this.timestamp = timestamp; } diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java index 254e1bf98..275aebd3d 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -24,9 +24,8 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; @@ -52,15 +51,23 @@ import java.util.Objects; import java.util.Properties; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; @@ -68,9 +75,11 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; -import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopics; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateScanBoundedMode; /** Upsert-Kafka factory. */ public class UpsertKafkaDynamicTableFactory @@ -87,7 +96,6 @@ public String factoryIdentifier() { public Set> requiredOptions() { final Set> options = new HashSet<>(); options.add(PROPS_BOOTSTRAP_SERVERS); - options.add(TOPIC); options.add(KEY_FORMAT); options.add(VALUE_FORMAT); return options; @@ -96,14 +104,27 @@ public Set> requiredOptions() { @Override public Set> optionalOptions() { final Set> options = new HashSet<>(); + options.add(TOPIC); + options.add(TOPIC_PATTERN); options.add(KEY_FIELDS_PREFIX); options.add(VALUE_FIELDS_INCLUDE); options.add(SINK_PARALLELISM); options.add(SINK_BUFFER_FLUSH_INTERVAL); options.add(SINK_BUFFER_FLUSH_MAX_ROWS); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); + options.add(SCAN_PARALLELISM); return options; } + @Override + public Set> forwardOptions() { + return Stream.of(DELIVERY_GUARANTEE, TRANSACTIONAL_ID_PREFIX).collect(Collectors.toSet()); + } + @Override public DynamicTableSource createDynamicTableSource(Context context) { FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); @@ -129,6 +150,10 @@ public DynamicTableSource createDynamicTableSource(Context context) { // always use earliest to keep data integrity StartupMode earliest = StartupMode.EARLIEST; + final BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + + Integer parallelism = tableOptions.get(SCAN_PARALLELISM); + return new KafkaDynamicSource( context.getPhysicalRowDataType(), keyDecodingFormat, @@ -136,17 +161,18 @@ public DynamicTableSource createDynamicTableSource(Context context) { keyValueProjections.f0, keyValueProjections.f1, keyPrefix, - getSourceTopics(tableOptions), - getSourceTopicPattern(tableOptions), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, earliest, Collections.emptyMap(), 0, - BoundedMode.UNBOUNDED, - Collections.emptyMap(), - 0, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, true, - context.getObjectIdentifier().asSummaryString()); + context.getObjectIdentifier().asSummaryString(), + parallelism); } @Override @@ -169,6 +195,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyEncodingFormat, valueEncodingFormat, context.getPrimaryKeyIndexes()); + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); Tuple2 keyValueProjections = createKeyValueProjections(context.getCatalogTable()); @@ -192,10 +219,11 @@ public DynamicTableSink createDynamicTableSink(Context context) { keyValueProjections.f0, keyValueProjections.f1, keyPrefix, - tableOptions.get(TOPIC).get(0), + getTopics(tableOptions), + getTopicPattern(tableOptions), properties, null, - DeliveryGuarantee.AT_LEAST_ONCE, + tableOptions.get(DELIVERY_GUARANTEE), true, flushMode, parallelism, @@ -227,7 +255,7 @@ private static void validateSource( Format keyFormat, Format valueFormat, int[] primaryKeyIndexes) { - validateTopic(tableOptions); + validateScanBoundedMode(tableOptions); validateFormat(keyFormat, valueFormat, tableOptions); validatePKConstraints(primaryKeyIndexes); } @@ -237,21 +265,11 @@ private static void validateSink( Format keyFormat, Format valueFormat, int[] primaryKeyIndexes) { - validateTopic(tableOptions); validateFormat(keyFormat, valueFormat, tableOptions); validatePKConstraints(primaryKeyIndexes); validateSinkBufferFlush(tableOptions); } - private static void validateTopic(ReadableConfig tableOptions) { - List topic = tableOptions.get(TOPIC); - if (topic.size() > 1) { - throw new ValidationException( - "The 'upsert-kafka' connector doesn't support topic list now. " - + "Please use single topic as the value of the parameter 'topic'."); - } - } - private static void validateFormat( Format keyFormat, Format valueFormat, ReadableConfig tableOptions) { if (!keyFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java index e2b428eec..cffdc8ea4 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/JSONKeyValueDeserializationSchema.java @@ -20,13 +20,12 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import static org.apache.flink.api.java.typeutils.TypeExtractor.getForClass; @@ -43,6 +42,7 @@ * (String) and "partition" (int). */ @PublicEvolving +@Deprecated public class JSONKeyValueDeserializationSchema implements KafkaDeserializationSchema { private static final long serialVersionUID = 1509391548173891955L; diff --git a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java index 1c3eaa620..05e0eaea1 100644 --- a/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java +++ b/flink-connector-kafka/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationKeyValueSerializationSchema.java @@ -41,6 +41,7 @@ * @param The value type to be serialized. */ @PublicEvolving +@Deprecated public class TypeInformationKeyValueSerializationSchema implements KafkaDeserializationSchema>, KeyedSerializationSchema> { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java b/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java new file mode 100644 index 000000000..26d080484 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/KafkaAssertjConfiguration.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink; + +import org.assertj.core.configuration.Configuration; + +/** Configuration for AssertJ that shows full stack traces for unmatched exceptions. */ +public class KafkaAssertjConfiguration extends Configuration { + public KafkaAssertjConfiguration() { + // in case of an assertion error, show the full stack trace + // for green builds, this is not changing anything + setMaxStackTraceElementsDisplayed(10000); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java new file mode 100644 index 000000000..edd867565 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/DynamicKafkaSourceITTest.java @@ -0,0 +1,834 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DynamicKafkaSourceExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.connector.kafka.testutils.TwoKafkaContainers; +import org.apache.flink.connector.kafka.testutils.YamlFileMetadataService; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.TestLogger; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration tests for {@link org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource}. + */ +public class DynamicKafkaSourceITTest extends TestLogger { + + private static final String TOPIC = "DynamicKafkaSourceITTest"; + private static final int NUM_PARTITIONS = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata1; + MiniClusterWithClientResource miniClusterResource; + static InMemoryReporter reporter; + + @TempDir File testDir; + + @Nested + @TestInstance(TestInstance.Lifecycle.PER_CLASS) + class DynamicKafkaSourceSpecificTests { + @BeforeAll + void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_PARTITIONS, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + kafkaClusterTestEnvMetadata1 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1); + } + + @BeforeEach + void beforeEach() throws Exception { + reporter = InMemoryReporter.create(); + miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(2) + .setConfiguration( + reporter.addToConfiguration(new Configuration())) + .build()); + miniClusterResource.before(); + } + + @AfterEach + void afterEach() { + reporter.close(); + miniClusterResource.after(); + } + + @AfterAll + void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testBasicMultiClusterRead() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + mockKafkaMetadataService.getAllStreams().stream() + .map(KafkaStream::getStreamId) + .collect(Collectors.toSet())) + .setKafkaMetadataService(mockKafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() + < DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT + && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range( + 0, + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + * NUM_PARTITIONS + * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testSingleClusterTopicMetadataService() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + Properties properties = new Properties(); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + + KafkaMetadataService kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds( + // use topics as stream ids + Collections.singleton(TOPIC)) + .setKafkaMetadataService(kafkaMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + CloseableIterator iterator = stream.executeAndCollect(); + List results = new ArrayList<>(); + while (results.size() < NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT && iterator.hasNext()) { + results.add(iterator.next()); + } + + iterator.close(); + + // check that all test records have been consumed + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testMigrationUsingFileMetadataService() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-file-metadata-service"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + AtomicInteger latestValueOffset = + new AtomicInteger( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0)); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + results.add(iterator.next()); + + // trigger metadata update to consume from two clusters + if (results.size() == NUM_RECORDS_PER_SPLIT) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(0), + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + + // trigger another metadata update to remove old cluster + if (results.size() == latestValueOffset.get()) { + latestValueOffset.set( + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset.get())); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata(1))); + } + } catch (NoSuchElementException e) { + // swallow and wait + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + // we will produce 3x + return results.size() == NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3; + }, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + } + + // verify no data loss / duplication in metadata changes + // cluster0 contains 0-10 + // cluster 1 contains 10-30 + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3) + .boxed() + .collect(Collectors.toList())); + } + + @Test + void testTopicReAddMigrationUsingFileMetadataService() throws Throwable { + // setup topics + int kafkaClusterIdx = 0; + String topic1 = "test-topic-re-add-1"; + String topic2 = "test-topic-re-add-2"; + DynamicKafkaSourceTestHelper.createTopic(kafkaClusterIdx, topic1, NUM_PARTITIONS); + DynamicKafkaSourceTestHelper.createTopic(kafkaClusterIdx, topic2, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-topic-re-add-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic1), + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + int stage1Records = + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, topic1, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + int stage2Records = + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, + topic2, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + stage1Records); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + results.add(iterator.next()); + + // switch to second topic after first is read + if (results.size() == stage1Records) { + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic2), + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + } + + // re-add first topic again after second is read + // produce another batch to first topic + if (results.size() == stage2Records) { + DynamicKafkaSourceTestHelper.produceToKafka( + kafkaClusterIdx, + topic1, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + stage2Records); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + ImmutableList.of(topic1, topic2), + ImmutableList.of( + DynamicKafkaSourceTestHelper + .getKafkaClusterTestEnvMetadata( + kafkaClusterIdx))); + } + } catch (NoSuchElementException e) { + // swallow and wait + } catch (IOException e) { + throw new UncheckedIOException(e); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + // first batch of topic 1 * 2 + topic 2 + second batch of topic 1 + return results.size() == NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 4; + }, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + } + + // verify data + Stream expectedFullRead = + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT * 3).boxed(); + Stream expectedReRead = + IntStream.range(0, NUM_PARTITIONS * NUM_RECORDS_PER_SPLIT).boxed(); + List expectedResults = + Stream.concat(expectedFullRead, expectedReRead).collect(Collectors.toList()); + assertThat(results).containsExactlyInAnyOrderElementsOf(expectedResults); + } + + @Test + void testStreamPatternSubscriber() throws Throwable { + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-1", NUM_PARTITIONS); + int lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, "stream-pattern-test-1", NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + DynamicKafkaSourceTestHelper.createTopic(0, "stream-pattern-test-2", NUM_PARTITIONS); + lastValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, + "stream-pattern-test-2", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + DynamicKafkaSourceTestHelper.createTopic(1, "stream-pattern-test-3", NUM_PARTITIONS); + final int totalRecords = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + "stream-pattern-test-3", + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + lastValueOffset); + + // create new metadata file to consume from 1 cluster + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + + Set kafkaStreams = + getKafkaStreams( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties(), + ImmutableSet.of("stream-pattern-test-1", "stream-pattern-test-2")); + + writeClusterMetadataToFile(metadataFile, kafkaStreams); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty(CommonClientConfigs.GROUP_ID_CONFIG, "dynamic-kafka-src"); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamPattern(Pattern.compile("stream-pattern-test-.+")) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + List results = new ArrayList<>(); + + try (CloseableIterator iterator = stream.executeAndCollect()) { + CommonTestUtils.waitUtil( + () -> { + try { + Integer record = iterator.next(); + results.add(record); + + // add third stream that matches the regex + if (results.size() == NUM_RECORDS_PER_SPLIT) { + kafkaStreams.add( + getKafkaStream( + kafkaClusterTestEnvMetadata1 + .getKafkaClusterId(), + kafkaClusterTestEnvMetadata1 + .getStandardProperties(), + "stream-pattern-test-3")); + writeClusterMetadataToFile(metadataFile, kafkaStreams); + } + } catch (NoSuchElementException e) { + // swallow + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return results.size() == totalRecords; + }, + Duration.ofSeconds(15), + "Could not obtain the required records within the timeout"); + } + // verify no data loss / duplication in metadata changes + assertThat(results) + .containsExactlyInAnyOrderElementsOf( + IntStream.range(0, totalRecords).boxed().collect(Collectors.toList())); + } + + @Test + void testMetricsLifecycleManagement() throws Throwable { + // setup topics on two clusters + String fixedTopic = "test-metrics-lifecycle-mgmt"; + DynamicKafkaSourceTestHelper.createTopic(fixedTopic, NUM_PARTITIONS); + + // Flink job config and env + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + env.setRestartStrategy(RestartStrategies.noRestart()); + Properties properties = new Properties(); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "1000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "5000"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + "2"); + properties.setProperty( + CommonClientConfigs.GROUP_ID_CONFIG, "testMetricsLifecycleManagement"); + + // create new metadata file to consume from 1 cluster + String testStreamId = "test-file-metadata-service-stream"; + File metadataFile = File.createTempFile(testDir.getPath() + "/metadata", ".yaml"); + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService(metadataFile.getPath(), Duration.ofMillis(100)); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0))); + + DynamicKafkaSource dynamicKafkaSource = + DynamicKafkaSource.builder() + .setStreamIds(Collections.singleton(testStreamId)) + .setKafkaMetadataService(yamlFileMetadataService) + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly( + IntegerDeserializer.class)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setProperties(properties) + .build(); + + DataStreamSource stream = + env.fromSource( + dynamicKafkaSource, + WatermarkStrategy.noWatermarks(), + "dynamic-kafka-src"); + + int latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 0, fixedTopic, NUM_PARTITIONS, NUM_RECORDS_PER_SPLIT, 0); + List results = new ArrayList<>(); + try (CloseableIterator iterator = stream.executeAndCollect()) { + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + assertThat(results) + .containsOnlyOnceElementsOf( + IntStream.range(0, latestValueOffset) + .boxed() + .collect(Collectors.toList())); + + // should contain cluster 0 metrics + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + // setup test data for cluster 1 and stop consuming from cluster 0 + latestValueOffset = + DynamicKafkaSourceTestHelper.produceToKafka( + 1, + fixedTopic, + NUM_PARTITIONS, + NUM_RECORDS_PER_SPLIT, + latestValueOffset); + writeClusterMetadataToFile( + metadataFile, + testStreamId, + fixedTopic, + ImmutableList.of( + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(1))); + while (results.size() < latestValueOffset && iterator.hasNext()) { + results.add(iterator.next()); + } + + // cluster 0 is not being consumed from, metrics should not appear + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .doesNotContainPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-0.*")); + + assertThat(findMetrics(reporter, DYNAMIC_KAFKA_SOURCE_METRIC_GROUP)) + .allSatisfy( + metricName -> + assertThat(metricName) + .containsPattern( + ".*" + + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP + + "\\.kafkaCluster\\.kafka-cluster-1.*")); + } + } + + private void writeClusterMetadataToFile(File metadataFile, Set kafkaStreams) + throws IOException { + List streamMetadataList = new ArrayList<>(); + for (KafkaStream kafkaStream : kafkaStreams) { + List clusterMetadataList = + new ArrayList<>(); + + for (Map.Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + YamlFileMetadataService.StreamMetadata.ClusterMetadata clusterMetadata = + new YamlFileMetadataService.StreamMetadata.ClusterMetadata(); + clusterMetadata.setClusterId(entry.getKey()); + clusterMetadata.setBootstrapServers( + entry.getValue() + .getProperties() + .getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + clusterMetadata.setTopics(new ArrayList<>(entry.getValue().getTopics())); + clusterMetadataList.add(clusterMetadata); + } + + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(); + streamMetadata.setStreamId(kafkaStream.getStreamId()); + streamMetadata.setClusterMetadataList(clusterMetadataList); + streamMetadataList.add(streamMetadata); + } + + YamlFileMetadataService.saveToYaml(streamMetadataList, metadataFile); + } + + private void writeClusterMetadataToFile( + File metadataFile, + String streamId, + List topics, + List kafkaClusterTestEnvMetadataList) + throws IOException { + List clusterMetadata = + kafkaClusterTestEnvMetadataList.stream() + .map( + KafkaClusterTestEnvMetadata -> + new YamlFileMetadataService.StreamMetadata + .ClusterMetadata( + KafkaClusterTestEnvMetadata.getKafkaClusterId(), + KafkaClusterTestEnvMetadata + .getBrokerConnectionStrings(), + topics)) + .collect(Collectors.toList()); + YamlFileMetadataService.StreamMetadata streamMetadata = + new YamlFileMetadataService.StreamMetadata(streamId, clusterMetadata); + YamlFileMetadataService.saveToYaml( + Collections.singletonList(streamMetadata), metadataFile); + } + + private void writeClusterMetadataToFile( + File metadataFile, + String streamId, + String topic, + List kafkaClusterTestEnvMetadataList) + throws IOException { + writeClusterMetadataToFile( + metadataFile, + streamId, + ImmutableList.of(topic), + kafkaClusterTestEnvMetadataList); + } + + private Set findMetrics(InMemoryReporter inMemoryReporter, String groupPattern) { + Optional groups = inMemoryReporter.findGroup(groupPattern); + assertThat(groups).isPresent(); + return inMemoryReporter.getMetricsByGroup(groups.get()).keySet().stream() + .map(metricName -> groups.get().getMetricIdentifier(metricName)) + .collect(Collectors.toSet()); + } + + private Set getKafkaStreams( + String kafkaClusterId, Properties properties, Collection topics) { + return topics.stream() + .map(topic -> getKafkaStream(kafkaClusterId, properties, topic)) + .collect(Collectors.toSet()); + } + + private KafkaStream getKafkaStream( + String kafkaClusterId, Properties properties, String topic) { + return new KafkaStream( + topic, + Collections.singletonMap( + kafkaClusterId, + new ClusterMetadata(Collections.singleton(topic), properties))); + } + } + + /** Integration test based on connector testing framework. */ + @Nested + class IntegrationTests extends SourceTestSuiteBase { + @TestSemantics + CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + // Defines test environment on Flink MiniCluster + @SuppressWarnings("unused") + @TestEnv + MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + @TestExternalSystem + DefaultContainerizedExternalSystem twoKafkas = + DefaultContainerizedExternalSystem.builder() + .fromContainer(new TwoKafkaContainers()) + .build(); + + @SuppressWarnings("unused") + @TestContext + DynamicKafkaSourceExternalContextFactory twoClusters = + new DynamicKafkaSourceExternalContextFactory( + twoKafkas.getContainer().getKafka0(), + twoKafkas.getContainer().getKafka1(), + Collections.emptyList()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java new file mode 100644 index 000000000..66caec4c7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumStateSerializerTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.source.enumerator.AssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test {@link + * org.apache.flink.connector.kafka.dynamic.source.enumerator.DynamicKafkaSourceEnumStateSerializer}. + */ +public class DynamicKafkaSourceEnumStateSerializerTest { + + @Test + public void testSerde() throws Exception { + DynamicKafkaSourceEnumStateSerializer dynamicKafkaSourceEnumStateSerializer = + new DynamicKafkaSourceEnumStateSerializer(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster0:9092"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "cluster1:9092"); + + Set kafkaStreams = + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster1)))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = + new DynamicKafkaSourceEnumState( + kafkaStreams, + ImmutableMap.of( + "cluster0", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic0", 0), + AssignmentStatus.ASSIGNED), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic1", 1), + AssignmentStatus.UNASSIGNED_INITIAL)), + true), + "cluster1", + new KafkaSourceEnumState( + ImmutableSet.of( + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic2", 0), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic3", 1), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic4", 2), + AssignmentStatus.UNASSIGNED_INITIAL), + new TopicPartitionAndAssignmentStatus( + new TopicPartition("topic5", 3), + AssignmentStatus.UNASSIGNED_INITIAL)), + false))); + + DynamicKafkaSourceEnumState dynamicKafkaSourceEnumStateAfterSerde = + dynamicKafkaSourceEnumStateSerializer.deserialize( + 1, + dynamicKafkaSourceEnumStateSerializer.serialize( + dynamicKafkaSourceEnumState)); + + assertThat(dynamicKafkaSourceEnumState) + .usingRecursiveComparison() + .isEqualTo(dynamicKafkaSourceEnumStateAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java new file mode 100644 index 000000000..861333458 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/DynamicKafkaSourceEnumeratorTest.java @@ -0,0 +1,1065 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceOptions; +import org.apache.flink.connector.kafka.dynamic.source.GetMetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.enumerator.subscriber.KafkaStreamSetSubscriber; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.AssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.TopicPartitionAndAssignmentStatus; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** A test for {@link DynamicKafkaSourceEnumerator}. */ +public class DynamicKafkaSourceEnumeratorTest { + private static final int NUM_SUBTASKS = 3; + private static final String TOPIC = "DynamicKafkaSourceEnumeratorTest"; + private static final int NUM_SPLITS_PER_CLUSTER = 3; + private static final int NUM_RECORDS_PER_SPLIT = 5; + + @BeforeAll + public static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + } + + @AfterAll + public static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + public void testStartupWithoutContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithContinuousDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getOneTimeCallables()).isEmpty(); + assertThat(context.getPeriodicCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_noPeriodicDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, new MockKafkaMetadataService(true), (properties) -> {})) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + assertThatThrownBy(() -> runAllOneTimeCallables(context)) + .as( + "Exception expected since periodic discovery is disabled and metadata is required for setting up the job") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testStartupWithKafkaMetadataServiceFailure_withContinuousDiscovery() + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(true), + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .as("Exception expected since there is no state") + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void + testStartupWithKafkaMetadataServiceFailure_withContinuousDiscoveryAndCheckpointState() + throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService(true), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + // no exception + context.runPeriodicCallable(0); + + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + } + } + + @Test + public void testHandleMetadataServiceError() throws Throwable { + int failureThreshold = 5; + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_FAILURE_THRESHOLD.key(), + Integer.toString(failureThreshold)); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + + assertThat(context.getPeriodicCallables()).hasSize(1); + context.runPeriodicCallable(0); + + // init splits + runAllOneTimeCallables(context); + + // swap to exceptional metadata service + mockKafkaMetadataService.setThrowException(true); + + for (int i = 0; i < failureThreshold; i++) { + context.runPeriodicCallable(0); + } + + for (int i = 0; i < 2; i++) { + assertThatThrownBy(() -> context.runPeriodicCallable(0)) + .hasRootCause(new RuntimeException("Mock exception")); + // Need to reset internal throwable reference after each invocation of + // runPeriodicCallable, + // since the context caches the previous exceptions indefinitely + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(context, "errorInWorkerThread"); + errorInWorkerThread.set(null); + } + + mockKafkaMetadataService.setThrowException(false); + assertThatCode(() -> context.runPeriodicCallable(0)) + .as("Exception counter should have been reset") + .doesNotThrowAnyException(); + } + } + + @Test + public void testKafkaMetadataServiceDiscovery() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + mockKafkaMetadataService, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + context.runPeriodicCallable(0); + + // 1 callable for main enumerator and 2 for the sub enumerators since we have 2 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + int currentNumSplits = context.getSplitsAssignmentSequence().size(); + + // no changes to splits + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()).hasSize(currentNumSplits); + + // mock metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + + // changes should have occurred here + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "1 additional split assignment since there was 1 metadata update that caused a change") + .hasSize(currentNumSplits + 1); + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + } + } + + @Test + public void testReaderRegistrationAfterSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no splits should be assigned yet since there are no readers") + .isEmpty(); + assertThat(context.getSplitsAssignmentSequence()) + .as("no readers have registered yet") + .isEmpty(); + assertThat(context.getSentSourceEvent()).as("no readers have registered yet").isEmpty(); + + // initialize readers 0 and 2 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSentSourceEvent().keySet()) + .as("reader 0 and 2 should have only received the source event") + .containsExactlyInAnyOrder(0, 2); + Set allReadersThatReceivedSplits = + context.getSplitsAssignmentSequence().stream() + .flatMap( + splitAssignment -> + splitAssignment.assignment().keySet().stream()) + .collect(Collectors.toSet()); + assertThat(allReadersThatReceivedSplits) + .as("reader 0 and 2 should hve only received splits") + .containsExactlyInAnyOrder(0, 2); + + // initialize readers 1 + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + assertThat(context.getSentSourceEvent().keySet()) + .as("all readers should have received get metadata update event") + .containsExactlyInAnyOrder(0, 1, 2); + + for (List sourceEventsPerReader : context.getSentSourceEvent().values()) { + assertThat(sourceEventsPerReader) + .as("there should have been only 1 source event per reader") + .hasSize(1); + } + + // should have all splits assigned by now + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testReaderRegistrationBeforeSplitDiscovery() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + } + } + + @Test + public void testSnapshotState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + DynamicKafkaSourceEnumState stateBeforeSplitAssignment = enumerator.snapshotState(-1); + assertThat( + stateBeforeSplitAssignment.getClusterEnumeratorStates().values() + .stream() + .map(subState -> subState.assignedPartitions().stream()) + .count()) + .as("no readers registered, so state should be empty") + .isZero(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + assertThat(context.getSplitsAssignmentSequence()) + .as("readers should not be assigned yet since there are no splits") + .isEmpty(); + + // 1 callable for main enumerator and 3 for the sub enumerators since we have 3 clusters + runAllOneTimeCallables(context); + + assertThat(context.getOneTimeCallables()) + .as("There should be no more callables after running the 4") + .isEmpty(); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + DynamicKafkaSourceEnumState stateAfterSplitAssignment = enumerator.snapshotState(-1); + + assertThat( + stateAfterSplitAssignment.getClusterEnumeratorStates().values().stream() + .flatMap(enumState -> enumState.assignedPartitions().stream()) + .count()) + .isEqualTo( + NUM_SPLITS_PER_CLUSTER + * DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + } + } + + @Test + public void testEnumeratorStateDoesNotContainStaleTopicPartitions() throws Throwable { + final String topic2 = TOPIC + "_2"; + + DynamicKafkaSourceTestHelper.createTopic(topic2, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + topic2, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + + final Set initialStreams = + Collections.singleton( + new KafkaStream( + TOPIC, + DynamicKafkaSourceTestHelper.getClusterMetadataMap( + 0, TOPIC, topic2))); + + final Set updatedStreams = + Collections.singleton( + new KafkaStream( + TOPIC, + DynamicKafkaSourceTestHelper.getClusterMetadataMap(0, TOPIC))); + + try (MockKafkaMetadataService metadataService = + new MockKafkaMetadataService(initialStreams); + MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + metadataService, + (properties) -> + properties.setProperty( + DynamicKafkaSourceOptions + .STREAM_METADATA_DISCOVERY_INTERVAL_MS + .key(), + "1"))) { + enumerator.start(); + + context.runPeriodicCallable(0); + + runAllOneTimeCallables(context); + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + + DynamicKafkaSourceEnumState initialState = enumerator.snapshotState(-1); + + assertThat(getFilteredTopicPartitions(initialState, TOPIC, AssignmentStatus.ASSIGNED)) + .hasSize(2); + assertThat( + getFilteredTopicPartitions( + initialState, TOPIC, AssignmentStatus.UNASSIGNED_INITIAL)) + .hasSize(1); + assertThat(getFilteredTopicPartitions(initialState, topic2, AssignmentStatus.ASSIGNED)) + .hasSize(2); + assertThat( + getFilteredTopicPartitions( + initialState, topic2, AssignmentStatus.UNASSIGNED_INITIAL)) + .hasSize(1); + + // mock metadata change + metadataService.setKafkaStreams(updatedStreams); + + // changes should have occurred here + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + DynamicKafkaSourceEnumState migratedState = enumerator.snapshotState(-1); + + assertThat(getFilteredTopicPartitions(migratedState, TOPIC, AssignmentStatus.ASSIGNED)) + .hasSize(3); + assertThat( + getFilteredTopicPartitions( + migratedState, TOPIC, AssignmentStatus.UNASSIGNED_INITIAL)) + .isEmpty(); + assertThat(getFilteredTopicPartitions(migratedState, topic2, AssignmentStatus.ASSIGNED)) + .isEmpty(); + assertThat( + getFilteredTopicPartitions( + migratedState, topic2, AssignmentStatus.UNASSIGNED_INITIAL)) + .isEmpty(); + } + } + + @Test + public void testStartupWithCheckpointState() throws Throwable { + // init enumerator with checkpoint state + final DynamicKafkaSourceEnumState dynamicKafkaSourceEnumState = getCheckpointState(); + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + // start and check callables + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as( + "3 one time callables should have been scheduled. 1 for main enumerator and then 2 for each underlying enumerator") + .hasSize(1 + DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers and do split assignment + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + runAllOneTimeCallables(context); + + assertThat(context.getSentSourceEvent()).as("3 readers registered").hasSize(3); + for (List sourceEventsReceived : context.getSentSourceEvent().values()) { + assertThat(sourceEventsReceived) + .as("each reader should have sent 1 source event") + .hasSize(1); + } + + assertThat(context.getSplitsAssignmentSequence()) + .as( + "there should not be new splits and we don't assign previously assigned splits at startup and there is no metadata/split changes") + .isEmpty(); + } + + // test with periodic discovery enabled + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + new MockKafkaMetadataService( + Collections.singleton( + DynamicKafkaSourceTestHelper.getKafkaStream( + TOPIC))), + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + dynamicKafkaSourceEnumState, + new TestKafkaEnumContextProxyFactory())) { + enumerator.start(); + assertThat(context.getPeriodicCallables()) + .as("A periodic time partition discovery callable should have been scheduled") + .hasSize(1); + assertThat(context.getOneTimeCallables()) + .as( + "0 one time callables for main enumerator and 2 one time callables for each underlying enumerator should have been scheduled") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + // checkpoint state should have triggered split assignment + assertThat(context.getSplitsAssignmentSequence()) + .as( + "There is no split assignment since there are no new splits that are not contained in state") + .isEmpty(); + } + } + + @Test + public void testAddSplitsBack() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + runAllOneTimeCallables(context); + + Map> readerAssignmentsBeforeFailure = + getReaderAssignments(context); + assertThat(context.getSplitsAssignmentSequence()) + .as("we only expect splits have been assigned 2 times") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // simulate failures + context.unregisterReader(0); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(0), 0); + context.unregisterReader(2); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(2), 2); + assertThat(context.getSplitsAssignmentSequence()) + .as("Splits assignment should be unchanged") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS); + + // mock reader recovery + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + assertThat(getReaderAssignments(context)) + .containsAllEntriesOf(readerAssignmentsBeforeFailure); + assertThat(context.getSplitsAssignmentSequence()) + .as( + "the readers came back up, so there should be 2 additional split assignments in the sequence") + .hasSize(DynamicKafkaSourceTestHelper.NUM_KAFKA_CLUSTERS + 2); + } + } + + @Test + public void testEnumeratorDoesNotAssignDuplicateSplitsInMetadataUpdate() throws Throwable { + KafkaStream kafkaStreamWithOneCluster = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStreamWithOneCluster + .getClusterMetadataMap() + .remove(DynamicKafkaSourceTestHelper.getKafkaClusterId(1)); + + KafkaStream kafkaStreamWithTwoClusters = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + MockKafkaMetadataService mockKafkaMetadataService = + new MockKafkaMetadataService(Collections.singleton(kafkaStreamWithOneCluster)); + + Properties properties = new Properties(); + properties.setProperty( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "1"); + properties.setProperty(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + mockKafkaMetadataService, + context, + OffsetsInitializer.committedOffsets(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory())) { + + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + enumerator.start(); + + // run all discovery + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithOneCluster); + + // trigger metadata change + mockKafkaMetadataService.setKafkaStreams( + Collections.singleton(kafkaStreamWithTwoClusters)); + context.runPeriodicCallable(0); + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), kafkaStreamWithTwoClusters); + + Map splitAssignmentFrequencyMap = new HashMap<>(); + for (SplitsAssignment splitsAssignmentStep : + context.getSplitsAssignmentSequence()) { + for (List assignments : + splitsAssignmentStep.assignment().values()) { + for (DynamicKafkaSourceSplit assignment : assignments) { + splitAssignmentFrequencyMap.put( + assignment.splitId(), + splitAssignmentFrequencyMap.getOrDefault(assignment.splitId(), 0) + + 1); + } + } + } + + assertThat(splitAssignmentFrequencyMap.values()) + .as("all splits should have been assigned once") + .allMatch(count -> count == 1); + } + } + + @Test + public void testInitExceptionNonexistingKafkaCluster() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), + fakeProperties))))); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + @Test + public void testEnumeratorErrorPropagation() { + Properties fakeProperties = new Properties(); + fakeProperties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake-cluster:8080"); + KafkaStream fakeStream = + new KafkaStream( + "fake-stream", + Collections.singletonMap( + "fake-cluster", + new ClusterMetadata( + Collections.singleton("fake-topic"), fakeProperties))); + + MockKafkaMetadataService mockKafkaMetadataServiceWithUnavailableCluster = + new MockKafkaMetadataService( + ImmutableSet.of( + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC), fakeStream)); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator(context, mockKafkaMetadataServiceWithUnavailableCluster)) { + enumerator.start(); + + runAllOneTimeCallables(context); + } catch (Throwable throwable) { + assertThat(throwable).hasRootCauseInstanceOf(KafkaException.class); + } + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService) { + return createEnumerator(context, kafkaMetadataService, (properties) -> {}); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + Consumer applyPropertiesConsumer) { + return createEnumerator( + context, + new MockKafkaMetadataService( + Collections.singleton(DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC))), + applyPropertiesConsumer); + } + + private DynamicKafkaSourceEnumerator createEnumerator( + SplitEnumeratorContext context, + KafkaMetadataService kafkaMetadataService, + Consumer applyPropertiesConsumer) { + Properties properties = new Properties(); + applyPropertiesConsumer.accept(properties); + properties.putIfAbsent(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), "0"); + properties.putIfAbsent( + DynamicKafkaSourceOptions.STREAM_METADATA_DISCOVERY_INTERVAL_MS.key(), "0"); + return new DynamicKafkaSourceEnumerator( + new KafkaStreamSetSubscriber(Collections.singleton(TOPIC)), + kafkaMetadataService, + context, + OffsetsInitializer.earliest(), + new NoStoppingOffsetsInitializer(), + properties, + Boundedness.CONTINUOUS_UNBOUNDED, + new DynamicKafkaSourceEnumState(), + new TestKafkaEnumContextProxyFactory()); + } + + private void mockRegisterReaderAndSendReaderStartupEvent( + MockSplitEnumeratorContext context, + DynamicKafkaSourceEnumerator enumerator, + int reader) { + context.registerReader(new ReaderInfo(reader, "location " + reader)); + enumerator.addReader(reader); + // readers send source event at startup + enumerator.handleSourceEvent(reader, new GetMetadataUpdateEvent()); + } + + private void verifyAllSplitsHaveBeenAssigned( + List> splitsAssignmentSequence, + KafkaStream kafkaStream) { + Map> clusterTopicMap = new HashMap<>(); + for (Entry entry : + kafkaStream.getClusterMetadataMap().entrySet()) { + clusterTopicMap + .computeIfAbsent(entry.getKey(), unused -> new HashSet<>()) + .addAll(entry.getValue().getTopics()); + } + + Set splitsAssigned = + splitsAssignmentSequence.stream() + .flatMap( + splitsAssignment -> + splitsAssignment.assignment().values().stream() + .flatMap(Collection::stream)) + .collect(Collectors.toSet()); + + assertThat(splitsAssignmentSequence).isNotEmpty(); + + Map> clusterToTopicPartition = new HashMap<>(); + for (SplitsAssignment split : splitsAssignmentSequence) { + for (Entry> assignments : + split.assignment().entrySet()) { + for (DynamicKafkaSourceSplit assignment : assignments.getValue()) { + clusterToTopicPartition + .computeIfAbsent(assignment.getKafkaClusterId(), key -> new HashSet<>()) + .add(assignment.getKafkaPartitionSplit().getTopicPartition()); + } + } + } + + assertThat(splitsAssigned) + .hasSize(NUM_SPLITS_PER_CLUSTER * clusterTopicMap.keySet().size()); + + // verify correct clusters + for (String kafkaClusterId : clusterTopicMap.keySet()) { + assertThat(clusterToTopicPartition) + .as("All Kafka clusters must be assigned in the splits.") + .containsKey(kafkaClusterId); + } + + // verify topic partitions + Set assignedTopicPartitionSet = + clusterToTopicPartition.values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + for (Set topics : clusterTopicMap.values()) { + for (String topic : topics) { + Set expectedTopicPartitions = new HashSet<>(); + for (int i = 0; i < NUM_SPLITS_PER_CLUSTER; i++) { + expectedTopicPartitions.add(new TopicPartition(topic, i)); + } + assertThat(assignedTopicPartitionSet) + .as("splits must contain all topics and 2 partitions per topic") + .containsExactlyInAnyOrderElementsOf(expectedTopicPartitions); + } + } + } + + private Map> getReaderAssignments( + MockSplitEnumeratorContext context) { + Map> readerToSplits = new HashMap<>(); + for (SplitsAssignment split : + context.getSplitsAssignmentSequence()) { + for (Entry> assignments : + split.assignment().entrySet()) { + readerToSplits + .computeIfAbsent(assignments.getKey(), key -> new HashSet<>()) + .addAll(assignments.getValue()); + } + } + return readerToSplits; + } + + private List getFilteredTopicPartitions( + DynamicKafkaSourceEnumState state, String topic, AssignmentStatus assignmentStatus) { + return state.getClusterEnumeratorStates().values().stream() + .flatMap(s -> s.partitions().stream()) + .filter( + partition -> + partition.topicPartition().topic().equals(topic) + && partition.assignmentStatus() == assignmentStatus) + .map(TopicPartitionAndAssignmentStatus::topicPartition) + .collect(Collectors.toList()); + } + + private static void runAllOneTimeCallables(MockSplitEnumeratorContext context) + throws Throwable { + while (!context.getOneTimeCallables().isEmpty()) { + context.runNextOneTimeCallable(); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState(KafkaStream kafkaStream) + throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = + createEnumerator( + context, + new MockKafkaMetadataService(Collections.singleton(kafkaStream)), + (properties) -> {})) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned(context.getSplitsAssignmentSequence(), kafkaStream); + + return enumerator.snapshotState(-1); + } + } + + private DynamicKafkaSourceEnumState getCheckpointState() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + DynamicKafkaSourceEnumerator enumerator = createEnumerator(context)) { + enumerator.start(); + + // initialize all readers + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 0); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 1); + mockRegisterReaderAndSendReaderStartupEvent(context, enumerator, 2); + + runAllOneTimeCallables(context); + + verifyAllSplitsHaveBeenAssigned( + context.getSplitsAssignmentSequence(), + DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC)); + + return enumerator.snapshotState(-1); + } + } + + private static class TestKafkaEnumContextProxyFactory + implements StoppableKafkaEnumContextProxy.StoppableKafkaEnumContextProxyFactory { + + @Override + public StoppableKafkaEnumContextProxy create( + SplitEnumeratorContext enumContext, + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + Runnable signalNoMoreSplitsCallback) { + return new TestKafkaEnumContextProxy( + kafkaClusterId, + kafkaMetadataService, + (MockSplitEnumeratorContext) enumContext); + } + } + + private static class TestKafkaEnumContextProxy extends StoppableKafkaEnumContextProxy { + + private final SplitEnumeratorContext enumContext; + + public TestKafkaEnumContextProxy( + String kafkaClusterId, + KafkaMetadataService kafkaMetadataService, + MockSplitEnumeratorContext enumContext) { + super(kafkaClusterId, kafkaMetadataService, enumContext, null); + this.enumContext = enumContext; + } + + /** + * Schedule periodic callables under the coordinator executor, so we can use {@link + * MockSplitEnumeratorContext} to invoke the callable (split assignment) on demand to test + * the integration of KafkaSourceEnumerator. + */ + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + enumContext.callAsync( + wrapCallAsyncCallable(callable), + wrapCallAsyncCallableHandler(handler), + initialDelay, + period); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java new file mode 100644 index 000000000..e3dbf4fdf --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/enumerator/StoppableKafkaEnumContextProxyTest.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.enumerator; + +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.testutils.MockKafkaMetadataService; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.util.FlinkRuntimeException; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.common.errors.TimeoutException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.fail; + +/** A test for {@link StoppableKafkaEnumContextProxy}. */ +public class StoppableKafkaEnumContextProxyTest { + + private static final String ACTIVE_KAFKA_CLUSTER = "mock-kafka-cluster"; + private static final String INACTIVE_KAFKA_CLUSTER = "mock-inactive-kafka-cluster"; + + private volatile boolean throwExceptionFromMainCallable; + + @BeforeEach + public void beforeEach() { + throwExceptionFromMainCallable = true; + } + + @AfterAll + public static void afterAll() throws Exception {} + + @Test + public void testOneTimeCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + // not running the next periodic callable, since StoppableKafkaEnumContextProxy has the + // callable that periodically schedules the proxy task. So the proxy task (e.g. split + // discovery) is a one time callable in the context of source coordinator. + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableErrorHandling() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + enumContext.runNextOneTimeCallable(); + assertThat(isCallbackInvoked) + .as("callback should be skipped upon swallowing the error.") + .isFalse(); + } + } + + @Test + public void testPeriodicCallableThrowsExceptionOnActiveCluster() throws Throwable { + try (MockSplitEnumeratorContext enumContext = + new MockSplitEnumeratorContext<>(2); + StoppableKafkaEnumContextProxy enumContextProxy = + createStoppableKafkaEnumContextProxy(enumContext, ACTIVE_KAFKA_CLUSTER)) { + + AtomicBoolean isCallbackInvoked = new AtomicBoolean(); + setupKafkaTopicPartitionDiscoveryMockCallable(enumContextProxy, isCallbackInvoked); + + CommonTestUtils.waitUtil( + () -> enumContext.getOneTimeCallables().size() == 1, + Duration.ofSeconds(15), + "Could not schedule callable within timeout"); + + assertThatThrownBy(() -> runNextOneTimeCallableAndCatchException(enumContext)) + .isExactlyInstanceOf(FlinkRuntimeException.class); + assertThat(isCallbackInvoked) + .as("error callback should be invoked since splits have not been assigned yet.") + .isTrue(); + } + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext) { + return createStoppableKafkaEnumContextProxy(enumContext, INACTIVE_KAFKA_CLUSTER); + } + + private StoppableKafkaEnumContextProxy createStoppableKafkaEnumContextProxy( + SplitEnumeratorContext enumContext, String contextKafkaCluster) { + + KafkaStream mockStream = + new KafkaStream( + "mock-stream", + ImmutableMap.of( + ACTIVE_KAFKA_CLUSTER, + new ClusterMetadata( + ImmutableSet.of("mock-topic"), new Properties()))); + + return new StoppableKafkaEnumContextProxy( + contextKafkaCluster, + new MockKafkaMetadataService(Collections.singleton(mockStream)), + enumContext, + null); + } + + // this modeled after `KafkaSourceEnumerator` topic partition subscription to throw the same + // exceptions + private void setupKafkaTopicPartitionDiscoveryMockCallable( + StoppableKafkaEnumContextProxy enumContextProxy, AtomicBoolean isCallbackInvoked) { + enumContextProxy.callAsync( + () -> { + if (throwExceptionFromMainCallable) { + // mock Kafka Exception + throw new TimeoutException("Kafka server timed out"); + } else { + // ignore output + return null; + } + }, + (res, t) -> { + isCallbackInvoked.set(true); + if (t != null) { + throw new FlinkRuntimeException(t); + } + }, + 0, + 1000); + } + + private void runNextOneTimeCallableAndCatchException(MockSplitEnumeratorContext enumContext) + throws Throwable { + try { + enumContext.runNextOneTimeCallable(); + fail("TimeoutException should have been thrown"); + } catch (TimeoutException e) { + // catch only Kafka Timeout exceptions since it will be rethrown by + // `MockSplitEnumeratorContext` + AtomicReference errorInMainThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInMainThread"); + AtomicReference errorInWorkerThread = + (AtomicReference) + Whitebox.getInternalState(enumContext, "errorInWorkerThread"); + + assertThat(errorInMainThread.get()) + .as("Should be error in main executor thread for async io") + .isNotNull(); + assertThat(errorInWorkerThread.get()) + .as( + "Should not be error in worker thread that corresponds to source coordinator thread") + .isNull(); + } finally { + // reset MockSplitEnumeratorContext error state + Whitebox.setInternalState( + enumContext, "errorInMainThread", new AtomicReference()); + Whitebox.setInternalState( + enumContext, "errorInWorkerThread", new AtomicReference()); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java new file mode 100644 index 000000000..4e1fcf0ce --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metadata/SingleClusterTopicMetadataServiceTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metadata; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.metadata.SingleClusterTopicMetadataService; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestBase; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class SingleClusterTopicMetadataServiceTest { + + private static final String TOPIC0 = "SingleClusterTopicMetadataServiceTest-1"; + private static final String TOPIC1 = "SingleClusterTopicMetadataServiceTest-2"; + + private static KafkaMetadataService kafkaMetadataService; + private static KafkaTestBase.KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata0; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + DynamicKafkaSourceTestHelper.createTopic(TOPIC0, 3); + DynamicKafkaSourceTestHelper.createTopic(TOPIC1, 3); + + kafkaClusterTestEnvMetadata0 = + DynamicKafkaSourceTestHelper.getKafkaClusterTestEnvMetadata(0); + + kafkaMetadataService = + new SingleClusterTopicMetadataService( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + kafkaClusterTestEnvMetadata0.getStandardProperties()); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void getAllStreams() { + Set allStreams = kafkaMetadataService.getAllStreams(); + assertThat(allStreams) + .as("stream names should be equal to topic names") + .containsExactlyInAnyOrder( + new KafkaStream( + TOPIC0, + ImmutableMap.of( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC0), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))), + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties())))); + } + + @Test + void describeStreams() { + Map streamMap = + kafkaMetadataService.describeStreams(Collections.singleton(TOPIC1)); + assertThat(streamMap) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of( + TOPIC1, + new KafkaStream( + TOPIC1, + Collections.singletonMap( + kafkaClusterTestEnvMetadata0.getKafkaClusterId(), + new ClusterMetadata( + Collections.singleton(TOPIC1), + kafkaClusterTestEnvMetadata0 + .getStandardProperties()))))); + + assertThatCode( + () -> + kafkaMetadataService.describeStreams( + Collections.singleton("unknown-stream"))) + .as("the stream topic cannot be found in kafka and we rethrow") + .hasRootCauseInstanceOf(UnknownTopicOrPartitionException.class); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java new file mode 100644 index 000000000..71feeb8a7 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/metrics/KafkaClusterMetricGroupTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.metrics; + +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup.DYNAMIC_KAFKA_SOURCE_METRIC_GROUP; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.metrics.KafkaClusterMetricGroup}. + */ +public class KafkaClusterMetricGroupTest { + + private static MetricListener metricListener; + private static InternalSourceReaderMetricGroup mockInternalSourceReaderMetricGroup; + private static KafkaClusterMetricGroup kafkaClusterMetricGroup; + + @BeforeEach + public void beforeEach() { + metricListener = new MetricListener(); + mockInternalSourceReaderMetricGroup = + InternalSourceReaderMetricGroup.mock(metricListener.getMetricGroup()); + kafkaClusterMetricGroup = + new KafkaClusterMetricGroup( + mockInternalSourceReaderMetricGroup.addGroup( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP), + mockInternalSourceReaderMetricGroup, + "broker-bootstrap-server:443"); + } + + @Test + public void testGetAllVariables() { + // metric variables are wrapped in <...> + assertThat(kafkaClusterMetricGroup.getAllVariables()) + .as( + "variables should contain Kafka cluster info to distinguish multiple sub KafkaSourceReaders") + .containsEntry( + "<" + KafkaClusterMetricGroup.KAFKA_CLUSTER_GROUP_NAME + ">", + "broker-bootstrap-server:443"); + } + + @Test + public void testGetScopeComponents() { + assertThat(kafkaClusterMetricGroup.getScopeComponents()) + .as("scope components contains previously attached scope component") + .contains(DYNAMIC_KAFKA_SOURCE_METRIC_GROUP); + } + + @Test + public void testSetPendingRecordsGauge() { + kafkaClusterMetricGroup.setPendingRecordsGauge(() -> 5L); + + // these identifiers should be attached to distinguish distinguish multiple sub + // KafkaSourceReaders + Optional> pendingRecordsGauge = + metricListener.getGauge( + DYNAMIC_KAFKA_SOURCE_METRIC_GROUP, + "kafkaCluster", + "broker-bootstrap-server:443", + "pendingRecords"); + + assertThat(pendingRecordsGauge.get().getValue()).isEqualTo(5L); + } + + @Test + public void testGetIOMetricGroup() { + assertThat(kafkaClusterMetricGroup.getIOMetricGroup()) + .isEqualTo(mockInternalSourceReaderMetricGroup.getIOMetricGroup()); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java new file mode 100644 index 000000000..5094e0151 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/reader/DynamicKafkaSourceReaderTest.java @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.reader; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; +import org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplit; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.testutils.source.reader.SourceReaderTestBase; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; + +import com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.powermock.reflect.Whitebox; + +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.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Test for {@link org.apache.flink.connector.kafka.dynamic.source.reader.DynamicKafkaSourceReader}. + */ +public class DynamicKafkaSourceReaderTest extends SourceReaderTestBase { + private static final String TOPIC = "DynamicKafkaSourceReaderTest"; + + // we are testing two clusters and SourceReaderTestBase expects there to be a total of 10 splits + private static final int NUM_SPLITS_PER_CLUSTER = 5; + + private static String kafkaClusterId0; + private static String kafkaClusterId1; + + @BeforeAll + static void beforeAll() throws Throwable { + DynamicKafkaSourceTestHelper.setup(); + + DynamicKafkaSourceTestHelper.createTopic(TOPIC, NUM_SPLITS_PER_CLUSTER, 1); + DynamicKafkaSourceTestHelper.produceToKafka( + TOPIC, NUM_SPLITS_PER_CLUSTER, NUM_RECORDS_PER_SPLIT); + kafkaClusterId0 = DynamicKafkaSourceTestHelper.getKafkaClusterId(0); + kafkaClusterId1 = DynamicKafkaSourceTestHelper.getKafkaClusterId(1); + } + + @AfterAll + static void afterAll() throws Exception { + DynamicKafkaSourceTestHelper.tearDown(); + } + + @Test + void testHandleSourceEventWithRemovedMetadataAtStartup() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + // mock restoring state from Flink runtime + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + + // start reader + reader.start(); + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + + // remove cluster 0 + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + List splitsWithoutCluster0 = + splits.stream() + .filter(split -> !split.getKafkaClusterId().equals(kafkaClusterId0)) + .collect(Collectors.toList()); + assertThat(reader.snapshotState(-1)) + .as("The splits should not contain any split related to cluster 0") + .containsExactlyInAnyOrderElementsOf(splitsWithoutCluster0); + } + } + + @Test + void testNoSubReadersInputStatus() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + InputStatus inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.NOTHING_AVAILABLE, + inputStatus, + "nothing available since there are no sub readers created, there could be sub readers created in the future"); + + // notify that this reader will not be assigned anymore splits + reader.notifyNoMoreSplits(); + + inputStatus = reader.pollNext(readerOutput); + assertEquals( + InputStatus.END_OF_INPUT, + inputStatus, + "there will not be any more input from this reader since there are no splits"); + } + } + + @Test + void testNotifyNoMoreSplits() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + reader.start(); + + // Splits assigned + List splits = + getSplits(getNumSplits(), NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + reader.addSplits(splits); + + // Send no more splits + reader.notifyNoMoreSplits(); + + // Send metadata + MetadataUpdateEvent metadata = + DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC); + reader.handleSourceEvents(metadata); + + // Check consistency + InputStatus status; + do { + status = reader.pollNext(readerOutput); + } while (status != InputStatus.END_OF_INPUT); + + assertThat(readerOutput.getEmittedRecords()) + .hasSize(getNumSplits() * NUM_RECORDS_PER_SPLIT); + } + } + + @Test + void testAvailabilityFutureUpdates() throws Exception { + TestingReaderContext context = new TestingReaderContext(); + try (DynamicKafkaSourceReader reader = createReaderWithoutStart(context)) { + CompletableFuture futureAtInit = reader.isAvailable(); + assertThat(reader.isActivelyConsumingSplits()).isFalse(); + assertThat(futureAtInit) + .as("future is not complete at fresh startup since no readers are created") + .isNotDone(); + assertThat(getAvailabilityHelperSize(reader)).isZero(); + + reader.start(); + MetadataUpdateEvent metadata = + DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC); + reader.handleSourceEvents(metadata); + List splits = + getSplits( + getNumSplits(), + NUM_RECORDS_PER_SPLIT, + Boundedness.CONTINUOUS_UNBOUNDED); + reader.addSplits(splits); + CompletableFuture futureAfterSplitAssignment = reader.isAvailable(); + + assertThat(futureAtInit) + .as( + "New future should have been produced since metadata triggers reader creation") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(2); + + // remove cluster 0 + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + kafkaStream.getClusterMetadataMap().remove(kafkaClusterId0); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + + CompletableFuture futureAfterRemovingCluster0 = reader.isAvailable(); + assertThat(futureAfterRemovingCluster0) + .as("There should new future since the metadata has changed") + .isNotSameAs(futureAfterSplitAssignment); + assertThat(getAvailabilityHelperSize(reader)).isEqualTo(1); + } + } + + private int getAvailabilityHelperSize(DynamicKafkaSourceReader reader) { + return ((CompletableFuture[]) + Whitebox.getInternalState( + reader.getAvailabilityHelper(), "futuresToCombine")) + .length; + } + + @Test + void testReaderMetadataChangeWhenOneTopicChanges() throws Exception { + try (DynamicKafkaSourceReader reader = + (DynamicKafkaSourceReader) createReader()) { + + // splits with offsets + DynamicKafkaSourceSplit cluster0Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(0), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + DynamicKafkaSourceSplit cluster1Split = + new DynamicKafkaSourceSplit( + DynamicKafkaSourceTestHelper.getKafkaClusterId(1), + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + reader.addSplits(ImmutableList.of(cluster0Split, cluster1Split)); + + // metadata change with a topic changing + KafkaStream kafkaStream = DynamicKafkaSourceTestHelper.getKafkaStream(TOPIC); + Set topicsForCluster1 = + kafkaStream.getClusterMetadataMap().get(kafkaClusterId1).getTopics(); + topicsForCluster1.clear(); + topicsForCluster1.add("new topic"); + reader.handleSourceEvents(new MetadataUpdateEvent(Collections.singleton(kafkaStream))); + // same split but earlier offset + DynamicKafkaSourceSplit newCluster0Split = + new DynamicKafkaSourceSplit( + kafkaClusterId0, + new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 10)); + // new split + DynamicKafkaSourceSplit newCluster1Split = + new DynamicKafkaSourceSplit( + kafkaClusterId1, + new KafkaPartitionSplit(new TopicPartition("new topic", 0), 10)); + reader.addSplits(ImmutableList.of(newCluster0Split, newCluster1Split)); + + List assignedSplits = reader.snapshotState(-1); + + assertThat(assignedSplits) + .as( + "The new split for cluster 1 should be assigned and split for cluster 0 should retain offset 10") + .containsExactlyInAnyOrder(cluster0Split, newCluster1Split); + } + } + + @Override + protected SourceReader createReader() { + TestingReaderContext context = new TestingReaderContext(); + return startReader(createReaderWithoutStart(context), context); + } + + private DynamicKafkaSourceReader createReaderWithoutStart( + TestingReaderContext context) { + Properties properties = getRequiredProperties(); + return new DynamicKafkaSourceReader<>( + context, + KafkaRecordDeserializationSchema.valueOnly(IntegerDeserializer.class), + properties); + } + + private SourceReader startReader( + DynamicKafkaSourceReader reader, TestingReaderContext context) { + reader.start(); + assertThat(context.getSentEvents()) + .as("Reader sends GetMetadataUpdateEvent at startup") + .hasSize(1); + reader.handleSourceEvents(DynamicKafkaSourceTestHelper.getMetadataUpdateEvent(TOPIC)); + return reader; + } + + private static Properties getRequiredProperties() { + Properties properties = new Properties(); + properties.setProperty( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + properties.setProperty( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName()); + return properties; + } + + @Override + protected List getSplits( + int numSplits, int numRecordsPerSplit, Boundedness boundedness) { + List splits = new ArrayList<>(); + for (int i = 0; i < numSplits; i++) { + splits.add(getSplit(i, numRecordsPerSplit, boundedness)); + } + return splits; + } + + @Override + protected DynamicKafkaSourceSplit getSplit( + int splitId, int numRecords, Boundedness boundedness) { + long stoppingOffset = + boundedness == Boundedness.BOUNDED + ? NUM_RECORDS_PER_SPLIT + : KafkaPartitionSplit.NO_STOPPING_OFFSET; + + String kafkaClusterId; + int splitIdForCluster = splitId % NUM_SPLITS_PER_CLUSTER; + if (splitId < NUM_SPLITS_PER_CLUSTER) { + kafkaClusterId = "kafka-cluster-0"; + } else { + kafkaClusterId = "kafka-cluster-1"; + } + + return new DynamicKafkaSourceSplit( + kafkaClusterId, + new KafkaPartitionSplit( + new TopicPartition(TOPIC, splitIdForCluster), 0L, stoppingOffset)); + } + + @Override + protected long getNextRecordIndex(DynamicKafkaSourceSplit split) { + return split.getKafkaPartitionSplit().getStartingOffset(); + } + + private Map> splitsToClusterTopicMap(List splits) { + Map> clusterTopicMap = new HashMap<>(); + + for (DynamicKafkaSourceSplit split : splits) { + Set topics = + clusterTopicMap.computeIfAbsent( + split.getKafkaClusterId(), (ignore) -> new HashSet<>()); + topics.add(split.getKafkaPartitionSplit().getTopic()); + } + + return clusterTopicMap; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java new file mode 100644 index 000000000..4125219a8 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/dynamic/source/split/DynamicKafkaSourceSplitSerializerTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.dynamic.source.split; + +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; + +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * A test for {@link + * org.apache.flink.connector.kafka.dynamic.source.split.DynamicKafkaSourceSplitSerializer}. + */ +public class DynamicKafkaSourceSplitSerializerTest { + + @Test + public void testSerde() throws IOException { + DynamicKafkaSourceSplitSerializer serializer = new DynamicKafkaSourceSplitSerializer(); + DynamicKafkaSourceSplit dynamicKafkaSourceSplit = + new DynamicKafkaSourceSplit( + "test-cluster", + new KafkaPartitionSplit(new TopicPartition("test-topic", 3), 1)); + DynamicKafkaSourceSplit dynamicKafkaSourceSplitAfterSerde = + serializer.deserialize(1, serializer.serialize(dynamicKafkaSourceSplit)); + assertEquals(dynamicKafkaSourceSplit, dynamicKafkaSourceSplitAfterSerde); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java new file mode 100644 index 000000000..869399896 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/lineage/LineageUtilTest.java @@ -0,0 +1,74 @@ +package org.apache.flink.connector.kafka.lineage; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageDatasetFacet; +import org.apache.flink.streaming.api.lineage.SourceLineageVertex; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link LineageUtil}. */ +public class LineageUtilTest { + @Test + public void testSourceLineageVertexOf() { + LineageDataset dataset = new TestingLineageDataset(); + SourceLineageVertex sourceLineageVertex = + LineageUtil.sourceLineageVertexOf(Collections.singletonList(dataset)); + + assertThat(sourceLineageVertex.boundedness()).isEqualTo(Boundedness.CONTINUOUS_UNBOUNDED); + assertThat(sourceLineageVertex.datasets()).containsExactly(dataset); + } + + @Test + public void testDatasetNamespaceOf() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host"); + } + + @Test + public void testDatasetNamespaceOfWithSemicolon() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host1;my-kafka-host2"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host1"); + } + + @Test + public void testDatasetNamespaceOfWithComma() { + Properties properties = new Properties(); + properties.put("bootstrap.servers", "my-kafka-host1,my-kafka-host2"); + + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://my-kafka-host1"); + } + + @Test + public void testDatasetNamespaceWhenNoBootstrapServersProperty() { + Properties properties = new Properties(); + assertThat(LineageUtil.namespaceOf(properties)).isEqualTo("kafka://"); + } + + private static class TestingLineageDataset implements LineageDataset { + @Override + public String name() { + return null; + } + + @Override + public String namespace() { + return null; + } + + @Override + public Map facets() { + return null; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java new file mode 100644 index 000000000..45fc9aa17 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/ExactlyOnceKafkaWriterITCase.java @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ProducerPoolImpl; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.TestLoggerExtension; + +import com.google.common.collect.Iterables; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Properties; + +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; + +/** Tests for the standalone KafkaWriter. */ +@ExtendWith(TestLoggerExtension.class) +public class ExactlyOnceKafkaWriterITCase extends KafkaWriterTestBase { + + @Test + void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + + // test flush + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + SinkInitContext sinkInitContext = + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + + final KafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .as("the exception is not thrown again") + .doesNotThrowAnyException(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + + // async exception is checked and thrown on close + assertThatCode(writer::close).hasRootCauseInstanceOf(ProducerFencedException.class); + } + + @Test + void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { + Properties properties = getKafkaClientConfiguration(); + + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + final KafkaWriter writer = + createWriter( + DeliveryGuarantee.EXACTLY_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + + triggerProducerException(writer, properties); + // to ensure that the exceptional send request has completed + writer.getCurrentProducer().flush(); + + // test flush + assertThatCode(writer::close) + .as("flush should throw the exception from the WriterCallback") + .hasRootCauseExactlyInstanceOf(ProducerFencedException.class); + assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + } + + private void triggerProducerException(KafkaWriter writer, Properties properties) + throws IOException { + final String transactionalId = writer.getCurrentProducer().getTransactionalId(); + + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + producer.initTransactions(); + producer.beginTransaction(); + producer.send(new ProducerRecord<>(topic, "1".getBytes())); + producer.commitTransaction(); + } + + writer.write(1, SINK_WRITER_CONTEXT); + } + + /** Test that producer is not accidentally recreated or pool is used. */ + @Test + void testLingeringTransaction() throws Exception { + final KafkaWriter failedWriter = createWriter(DeliveryGuarantee.EXACTLY_ONCE); + + // create two lingering transactions + failedWriter.flush(false); + failedWriter.prepareCommit(); + failedWriter.snapshotState(1); + failedWriter.flush(false); + failedWriter.prepareCommit(); + failedWriter.snapshotState(2); + + try (final KafkaWriter recoveredWriter = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + recoveredWriter.write(1, SINK_WRITER_CONTEXT); + + recoveredWriter.flush(false); + Collection committables = recoveredWriter.prepareCommit(); + recoveredWriter.snapshotState(1); + assertThat(committables).hasSize(1); + final KafkaCommittable committable = committables.stream().findFirst().get(); + assertThat(committable.getProducer().isPresent()).isTrue(); + + committable.getProducer().get().commitTransaction(); + + List> records = + drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); + assertThat(records).hasSize(1); + } + + failedWriter.close(); + } + + /** Test that producers are reused when committed. */ + @Test + void usePooledProducerForTransactional() throws Exception { + try (final ExactlyOnceKafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(getProducers(writer)).hasSize(0); + + writer.write(1, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables0 = writer.prepareCommit(); + writer.snapshotState(1); + assertThat(committables0).hasSize(1); + final KafkaCommittable committable = committables0.stream().findFirst().get(); + assertThat(committable.getProducer().isPresent()).isTrue(); + + FlinkKafkaInternalProducer firstProducer = committable.getProducer().get(); + assertThat(firstProducer != writer.getCurrentProducer()) + .as("Expected different producer") + .isTrue(); + + // recycle first producer, KafkaCommitter would commit it and then return it + assertThat(getProducers(writer)).hasSize(0); + firstProducer.commitTransaction(); + try (WritableBackchannel backchannel = getBackchannel(writer)) { + backchannel.send(firstProducer.getTransactionalId()); + } + + writer.write(1, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables1 = writer.prepareCommit(); + writer.snapshotState(2); + assertThat(committables1).hasSize(1); + final KafkaCommittable committable1 = committables1.stream().findFirst().get(); + assertThat(committable1.getProducer().isPresent()).isTrue(); + + assertThat(firstProducer == writer.getCurrentProducer()) + .as("Expected recycled producer") + .isTrue(); + } + } + + /** + * Tests that if a pre-commit attempt occurs on an empty transaction, the writer should not emit + * a KafkaCommittable, and instead immediately commit the empty transaction and recycle the + * producer. + */ + @Test + void prepareCommitForEmptyTransaction() throws Exception { + try (final ExactlyOnceKafkaWriter writer = + createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(getProducers(writer)).hasSize(0); + + // no data written to current transaction + writer.flush(false); + Collection emptyCommittables = writer.prepareCommit(); + + assertThat(emptyCommittables).hasSize(0); + assertThat(getProducers(writer)).hasSize(1); + final FlinkKafkaInternalProducer recycledProducer = + Iterables.getFirst(getProducers(writer), null); + assertThat(recycledProducer.isInTransaction()).isFalse(); + } + } + + /** + * Tests that open transactions are automatically aborted on close such that successive writes + * succeed. + */ + @Test + void testAbortOnClose() throws Exception { + Properties properties = getKafkaClientConfiguration(); + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + writer.write(1, SINK_WRITER_CONTEXT); + assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); + } + + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.EXACTLY_ONCE)) { + writer.write(2, SINK_WRITER_CONTEXT); + writer.flush(false); + Collection committables = writer.prepareCommit(); + writer.snapshotState(1L); + + // manually commit here, which would only succeed if the first transaction was aborted + assertThat(committables).hasSize(1); + final KafkaCommittable committable = committables.stream().findFirst().get(); + String transactionalId = committable.getTransactionalId(); + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); + producer.commitTransaction(); + } + + assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1); + } + } + + private static Collection> getProducers( + ExactlyOnceKafkaWriter writer) { + return ((ProducerPoolImpl) writer.getProducerPool()).getProducers(); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java index 51770f033..4da573fb4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducerITCase.java @@ -17,10 +17,9 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -28,27 +27,28 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; import java.time.Duration; +import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -56,23 +56,25 @@ @ExtendWith(TestLoggerExtension.class) class FlinkKafkaInternalProducerITCase { - private static final Logger LOG = - LoggerFactory.getLogger(FlinkKafkaInternalProducerITCase.class); - @Container private static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); + createKafkaContainer(FlinkKafkaInternalProducerITCase.class).withEmbeddedZookeeper(); - private static final String TRANSACTION_PREFIX = "test-transaction-"; + @AfterEach + public void check() { + checkProducerLeak(); + } @Test - void testInitTransactionId() { + void testResetTransactional() { final String topic = "test-init-transactions"; + final String transactionIdPrefix = "testInitTransactionId-"; try (FlinkKafkaInternalProducer reuse = new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { int numTransactions = 20; for (int i = 1; i <= numTransactions; i++) { - reuse.initTransactionId(TRANSACTION_PREFIX + i); + reuse.setTransactionId(transactionIdPrefix + i); + reuse.initTransactions(); reuse.beginTransaction(); reuse.send(new ProducerRecord<>(topic, "test-value-" + i)); if (i % 2 == 0) { @@ -81,12 +83,59 @@ void testInitTransactionId() { reuse.flush(); reuse.abortTransaction(); } - assertNumTransactions(i); + assertNumTransactions(i, transactionIdPrefix); assertThat(readRecords(topic).count()).isEqualTo(i / 2); } } } + @Test + void testCommitResumedTransaction() { + final String topic = "test-commit-resumed-transaction"; + final String transactionIdPrefix = "testCommitResumedTransaction-"; + final String transactionalId = transactionIdPrefix + "id"; + + KafkaCommittable snapshottedCommittable; + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { + producer.initTransactions(); + producer.beginTransaction(); + producer.send(new ProducerRecord<>(topic, "test-value")); + producer.flush(); + snapshottedCommittable = KafkaCommittable.of(producer); + } + + try (FlinkKafkaInternalProducer resumedProducer = + new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { + resumedProducer.resumeTransaction( + snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); + resumedProducer.commitTransaction(); + } + + assertNumTransactions(1, transactionIdPrefix); + assertThat(readRecords(topic).count()).isEqualTo(1); + } + + @Test + void testCommitResumedEmptyTransactionShouldFail() { + KafkaCommittable snapshottedCommittable; + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { + producer.initTransactions(); + producer.beginTransaction(); + snapshottedCommittable = KafkaCommittable.of(producer); + } + + try (FlinkKafkaInternalProducer resumedProducer = + new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { + resumedProducer.resumeTransaction( + snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); + + assertThatThrownBy(resumedProducer::commitTransaction) + .isInstanceOf(InvalidTxnStateException.class); + } + } + @ParameterizedTest @MethodSource("provideTransactionsFinalizer") void testResetInnerTransactionIfFinalizingTransactionFailed( @@ -126,15 +175,15 @@ private static Properties getProperties() { } private static List>> provideTransactionsFinalizer() { - return Lists.newArrayList( + return Arrays.asList( FlinkKafkaInternalProducer::commitTransaction, FlinkKafkaInternalProducer::abortTransaction); } - private void assertNumTransactions(int numTransactions) { + private void assertNumTransactions(int numTransactions, String transactionIdPrefix) { List transactions = new KafkaTransactionLog(getProperties()) - .getTransactions(id -> id.startsWith(TRANSACTION_PREFIX)); + .getTransactions(id -> id.startsWith(transactionIdPrefix)); assertThat( transactions.stream() .map(KafkaTransactionLog.TransactionRecord::getTransactionId) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java index ea9d893ed..cb27bf2c3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java @@ -18,98 +18,128 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.connector.sink2.mocks.MockCommitRequest; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; +import org.apache.flink.connector.kafka.sink.internal.ReadableBackchannel; import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.Condition; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; -import java.time.Duration; import java.util.Collections; import java.util.Properties; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaCommitter}. */ @ExtendWith({TestLoggerExtension.class}) -public class KafkaCommitterTest { +class KafkaCommitterTest { private static final int PRODUCER_ID = 0; private static final short EPOCH = 0; - private static final String TRANSACTIONAL_ID = "transactionalId"; + private static final String TRANS_ID = "transactionalId"; + public static final int ATTEMPT = 2; + public static final int SUB_ID = 1; + + @AfterEach + public void check() { + checkProducerLeak(); + } /** Causes a network error by inactive broker and tests that a retry will happen. */ @Test public void testRetryCommittableOnRetriableError() throws IOException, InterruptedException { Properties properties = getProperties(); - try (final KafkaCommitter committer = new KafkaCommitter(properties); + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID); - Recyclable> recyclable = - new Recyclable<>(producer, p -> {})) { + new FlinkKafkaInternalProducer<>(properties, TRANS_ID); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { final MockCommitRequest request = - new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new MockCommitRequest<>(KafkaCommittable.of(producer)); producer.resumeTransaction(PRODUCER_ID, EPOCH); committer.commit(Collections.singletonList(request)); assertThat(request.getNumberOfRetries()).isEqualTo(1); - assertThat(recyclable.isRecycled()).isFalse(); - // FLINK-25531: force the producer to close immediately, else it would take 1 hour - producer.close(Duration.ZERO); + assertThat(backchannel).doesNotHave(recycledProducer()); } } @Test public void testFailJobOnUnknownFatalError() throws IOException, InterruptedException { Properties properties = getProperties(); - try (final KafkaCommitter committer = new KafkaCommitter(properties); + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, TRANSACTIONAL_ID); - Recyclable> recyclable = - new Recyclable<>(producer, p -> {})) { + new FlinkKafkaInternalProducer<>(properties, TRANS_ID); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { // will fail because transaction not started final MockCommitRequest request = - new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new MockCommitRequest<>(KafkaCommittable.of(producer)); committer.commit(Collections.singletonList(request)); assertThat(request.getFailedWithUnknownReason()) .isInstanceOf(IllegalStateException.class); assertThat(request.getFailedWithUnknownReason().getMessage()) .contains("Transaction was not started"); - assertThat(recyclable.isRecycled()).isTrue(); + assertThat(backchannel).doesNotHave(recycledProducer()); } } @Test - public void testKafkaCommitterClosesProducer() throws IOException, InterruptedException { + public void testFailJobOnKnownFatalError() throws IOException, InterruptedException { Properties properties = getProperties(); - FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer(properties, TRANSACTIONAL_ID) { - @Override - public void commitTransaction() throws ProducerFencedException {} - - @Override - public void flush() {} - }; - Recyclable> recyclable = - new Recyclable<>(producer, FlinkKafkaInternalProducer::close); - try (final KafkaCommitter committer = new KafkaCommitter(properties)) { + try (final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); + FlinkKafkaInternalProducer producer = + new MockProducer(properties, new ProducerFencedException("test")); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { + // will fail because transaction not started + final MockCommitRequest request = + new MockCommitRequest<>(KafkaCommittable.of(producer)); + committer.commit(Collections.singletonList(request)); + // do not recycle if a fail-over is triggered; + // else there may be a race-condition in creating a new transaction with the same name + assertThat(backchannel).has(recycledProducer()); + } + } + + @Test + public void testKafkaCommitterRecyclesProducer() throws IOException, InterruptedException { + Properties properties = getProperties(); + try (FlinkKafkaInternalProducer producer = new MockProducer(properties, null); + final KafkaCommitter committer = + new KafkaCommitter(properties, TRANS_ID, SUB_ID, ATTEMPT); + ReadableBackchannel backchannel = + BackchannelFactory.getInstance() + .getReadableBackchannel(SUB_ID, ATTEMPT, TRANS_ID)) { final MockCommitRequest request = new MockCommitRequest<>( - new KafkaCommittable(PRODUCER_ID, EPOCH, TRANSACTIONAL_ID, recyclable)); + new KafkaCommittable(PRODUCER_ID, EPOCH, TRANS_ID, producer)); committer.commit(Collections.singletonList(request)); - assertThat(recyclable.isRecycled()).isTrue(); - assertThat(producer.isClosed()).isTrue(); + assertThat(backchannel).has(recycledProducer()); } } + private Condition> recycledProducer() { + return new Condition<>(backchannel -> backchannel.poll() != null, "recycled producer"); + } + Properties getProperties() { Properties properties = new Properties(); properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "http://localhost:1"); @@ -119,4 +149,23 @@ Properties getProperties() { properties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); return properties; } + + private static class MockProducer extends FlinkKafkaInternalProducer { + private final RuntimeException commitException; + + public MockProducer(Properties properties, RuntimeException commitException) { + super(properties, KafkaCommitterTest.TRANS_ID); + this.commitException = commitException; + } + + @Override + public void commitTransaction() throws ProducerFencedException { + if (commitException != null) { + throw commitException; + } + } + + @Override + public void flush() {} + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java index 614624ea3..4d1437288 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java @@ -19,25 +19,38 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.Before; import org.junit.Test; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; @@ -51,6 +64,13 @@ public class KafkaRecordSerializationSchemaBuilderTest extends TestLogger { private static Map configurableConfiguration; private static Map configuration; + + private interface TestingTopicSelector + extends TopicSelector, KafkaDatasetIdentifierProvider {} + + private interface SerializationSchemaWithResultQueryable + extends SerializationSchema, ResultTypeQueryable {} + private static boolean isKeySerializer; @Before @@ -145,6 +165,27 @@ public void testSerializeRecordWithPartitioner() throws Exception { assertThat(opened.get()).isTrue(); } + @Test + public void testSerializeRecordWithHeaderProvider() throws Exception { + final HeaderProvider headerProvider = + (ignored) -> + new RecordHeaders( + Collections.singletonList(new RecordHeader("a", "a".getBytes()))); + + final KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopic(DEFAULT_TOPIC) + .setValueSerializationSchema(new SimpleStringSchema()) + .setHeaderProvider(headerProvider) + .build(); + final ProducerRecord record = schema.serialize("a", null, null); + assertThat(record).isNotNull(); + assertThat(record.headers()) + .singleElement() + .extracting(Header::key, Header::value) + .containsExactly("a", "a".getBytes(StandardCharsets.UTF_8)); + } + @Test public void testSerializeRecordWithKey() { final SerializationSchema serializationSchema = new SimpleStringSchema(); @@ -162,7 +203,7 @@ public void testSerializeRecordWithKey() { @Test public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -179,7 +220,7 @@ public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception @Test public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -193,7 +234,7 @@ public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exceptio @Test public void testSerializeRecordWithKafkaSerializer() throws Exception { - final Map config = ImmutableMap.of("configKey", "configValue"); + final Map config = Collections.singletonMap("configKey", "configValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() .setTopic(DEFAULT_TOPIC) @@ -233,6 +274,134 @@ public void testSerializeRecordWithTimestamp() { assertThat(recordWithInvalidTimestamp.timestamp()).isNull(); } + @Test + public void testGetLineageDatasetFacetsWhenTopicSelectorNotKafkaTopicsIdentifierProvider() { + SerializationSchema serializationSchema = new SimpleStringSchema(); + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector((TopicSelector) o -> DEFAULT_TOPIC) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + + assertThat(schema) + .asInstanceOf(InstanceOfAssertFactories.type(KafkaDatasetFacetProvider.class)) + .returns(Optional.empty(), KafkaDatasetFacetProvider::getKafkaDatasetFacet); + } + + @Test + public void testGetLineageDatasetFacetsWhenNoTopicsIdentifiersFound() { + SerializationSchema serializationSchema = new SimpleStringSchema(); + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.empty(); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + assertThat(schema) + .asInstanceOf(InstanceOfAssertFactories.type(KafkaDatasetFacetProvider.class)) + .returns(Optional.empty(), KafkaDatasetFacetProvider::getKafkaDatasetFacet); + } + + @Test + public void testGetLineageDatasetFacetsValueSerializationSchemaIsResultTypeQueryable() { + TypeInformation stringTypeInformation = TypeInformation.of(String.class); + SerializationSchemaWithResultQueryable serializationSchema = + new SerializationSchemaWithResultQueryable() { + + @Override + public TypeInformation getProducedType() { + return stringTypeInformation; + } + + @Override + public byte[] serialize(String o) { + return new byte[0]; + } + }; + + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + Arrays.asList("topic1", "topic2"))); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(serializationSchema) + .setKeySerializationSchema(serializationSchema) + .build(); + + Optional kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) schema).getKafkaDatasetFacet(); + + assertThat(kafkaDatasetFacet).isPresent(); + assertThat(kafkaDatasetFacet.get().getTopicIdentifier().getTopics()) + .containsExactly("topic1", "topic2"); + assertThat(((TypeDatasetFacetProvider) schema).getTypeDatasetFacet()) + .isPresent() + .get() + .extracting(TypeDatasetFacet::getTypeInformation) + .isEqualTo(stringTypeInformation); + } + + @Test + public void testGetLineageDatasetFacets() { + KafkaRecordSerializationSchema schema = + KafkaRecordSerializationSchema.builder() + .setTopicSelector( + new TestingTopicSelector() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics( + Arrays.asList("topic1", "topic2"))); + } + + @Override + public String apply(Object o) { + return DEFAULT_TOPIC; + } + }) + .setValueSerializationSchema(new SimpleStringSchema()) + .setKeySerializationSchema(new SimpleStringSchema()) + .build(); + + Optional kafkaDatasetFacet = + ((KafkaDatasetFacetProvider) schema).getKafkaDatasetFacet(); + + assertThat(kafkaDatasetFacet).isPresent(); + assertThat(kafkaDatasetFacet.get().getTopicIdentifier().getTopics()) + .containsExactly("topic1", "topic2"); + assertThat(((TypeDatasetFacetProvider) schema).getTypeDatasetFacet()) + .isPresent() + .get() + .extracting(TypeDatasetFacet::getTypeInformation) + .isEqualTo(BasicTypeInfo.STRING_TYPE_INFO); + } + private static void assertOnlyOneSerializerAllowed( List< Function< @@ -261,7 +430,7 @@ private static void assertOnlyOneSerializerAllowed( KafkaRecordSerializationSchemaBuilder, KafkaRecordSerializationSchemaBuilder>> valueSerializationSetter() { - return ImmutableList.of( + return Arrays.asList( (b) -> b.setKafkaValueSerializer(StringSerializer.class), (b) -> b.setValueSerializationSchema(new SimpleStringSchema()), (b) -> @@ -274,7 +443,7 @@ private static void assertOnlyOneSerializerAllowed( KafkaRecordSerializationSchemaBuilder, KafkaRecordSerializationSchemaBuilder>> keySerializationSetter() { - return ImmutableList.of( + return Arrays.asList( (b) -> b.setKafkaKeySerializer(StringSerializer.class), (b) -> b.setKeySerializationSchema(new SimpleStringSchema()), (b) -> diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java index eeecc84df..9e26cd4e1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.util.TestLogger; import org.apache.kafka.clients.producer.ProducerConfig; @@ -28,6 +29,7 @@ import java.util.function.Consumer; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link KafkaSinkBuilder}. */ public class KafkaSinkBuilderTest extends TestLogger { @@ -41,7 +43,7 @@ public class KafkaSinkBuilderTest extends TestLogger { }; @Test - public void testPropertyHandling() { + void testPropertyHandling() { validateProducerConfig( getBasicBuilder(), p -> { @@ -52,7 +54,6 @@ public void testPropertyHandling() { getBasicBuilder().setProperty("k1", "v1"), p -> { Arrays.stream(DEFAULT_KEYS).forEach(k -> assertThat(p).containsKey(k)); - p.containsKey("k1"); }); Properties testConf = new Properties(); @@ -78,7 +79,7 @@ public void testPropertyHandling() { } @Test - public void testBootstrapServerSetting() { + void testBootstrapServerSetting() { Properties testConf1 = new Properties(); testConf1.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "testServer"); @@ -87,6 +88,18 @@ public void testBootstrapServerSetting() { p -> assertThat(p).containsKeys(DEFAULT_KEYS)); } + @Test + void testTransactionIdSanityCheck() { + assertThatThrownBy( + () -> + getBasicBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .build()) + .isExactlyInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "EXACTLY_ONCE delivery guarantee requires a transactionIdPrefix to be set to provide unique transaction names across multiple KafkaSinks writing to the same Kafka cluster."); + } + private void validateProducerConfig( KafkaSinkBuilder builder, Consumer validator) { validator.accept(builder.build().getKafkaProducerConfig()); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index 942902f04..ba264764c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -17,21 +17,25 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.StateBackendOptions; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; @@ -42,25 +46,29 @@ import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; -import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.test.util.TestUtils; -import org.apache.flink.testutils.junit.SharedObjects; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.SharedObjectsExtension; import org.apache.flink.testutils.junit.SharedReference; -import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava30.com.google.common.base.Joiner; +import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; @@ -68,46 +76,50 @@ import org.apache.kafka.clients.admin.DeleteTopicsResult; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Nested; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; import org.testcontainers.utility.DockerImageName; import javax.annotation.Nullable; import java.io.File; -import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; 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.UUID; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.LongStream; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; /** Tests for using KafkaSink writing to a Kafka cluster. */ +@Testcontainers public class KafkaSinkITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); @@ -118,23 +130,27 @@ public class KafkaSinkITCase extends TestLogger { private static AdminClient admin; private String topic; - private SharedReference emittedRecordsCount; - private SharedReference emittedRecordsWithCheckpoint; - private SharedReference failed; - private SharedReference lastCheckpointedRecord; - @ClassRule + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(2) + .setNumberSlotsPerTaskManager(8) + .setConfiguration(new Configuration()) + .build()); + + @Container public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(KafkaSinkITCase.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); - @Rule public final SharedObjects sharedObjects = SharedObjects.create(); - - @Rule public final TemporaryFolder temp = new TemporaryFolder(); + @RegisterExtension + public final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); - @BeforeClass + @BeforeAll public static void setupAdmin() { Map properties = new HashMap<>(); properties.put( @@ -143,33 +159,29 @@ public static void setupAdmin() { admin = AdminClient.create(properties); } - @AfterClass + @AfterAll public static void teardownAdmin() { admin.close(); } - @Before - public void setUp() throws ExecutionException, InterruptedException, TimeoutException { - emittedRecordsCount = sharedObjects.add(new AtomicLong()); - emittedRecordsWithCheckpoint = sharedObjects.add(new AtomicLong()); - failed = sharedObjects.add(new AtomicBoolean(false)); - lastCheckpointedRecord = sharedObjects.add(new AtomicLong(0)); + @BeforeEach + public void setUp() throws ExecutionException, InterruptedException { topic = UUID.randomUUID().toString(); createTestTopic(topic, 1, TOPIC_REPLICATION_FACTOR); } - @After + @AfterEach public void tearDown() throws ExecutionException, InterruptedException, TimeoutException { + checkProducerLeak(); deleteTestTopic(topic); } /** Integration test based on connector testing framework. */ + @SuppressWarnings("unused") @Nested class IntegrationTests extends SinkTestSuiteBase { // Defines test environment on Flink MiniCluster - @SuppressWarnings("unused") - @TestEnv - MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); // Defines external system @TestExternalSystem @@ -180,14 +192,12 @@ class IntegrationTests extends SinkTestSuiteBase { DockerImageName.parse(DockerImageVersions.KAFKA))) .build(); - @SuppressWarnings("unused") @TestSemantics CheckpointingMode[] semantics = new CheckpointingMode[] { CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE }; - @SuppressWarnings("unused") @TestContext KafkaSinkExternalContextFactory sinkContext = new KafkaSinkExternalContextFactory(kafka.getContainer(), Collections.emptyList()); @@ -195,97 +205,108 @@ class IntegrationTests extends SinkTestSuiteBase { @Test public void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE, emittedRecordsCount); + writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE); } @Test public void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.NONE, emittedRecordsCount); + writeRecordsToKafka(DeliveryGuarantee.NONE); } - @Test - public void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws Exception { - writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, emittedRecordsWithCheckpoint); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testWriteRecordsToKafkaWithExactlyOnceGuarantee(boolean chained) throws Exception { + writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, chained); } @Test public void testRecoveryWithAtLeastOnceGuarantee() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.AT_LEAST_ONCE, - 1, - (records) -> assertThat(records).contains(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L)); + testRecoveryWithAssertion(DeliveryGuarantee.AT_LEAST_ONCE, 1); } - @Test - public void testRecoveryWithExactlyOnceGuarantee() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.EXACTLY_ONCE, - 1, - (records) -> - assertThat(records) - .contains( - (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)))); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testRecoveryWithExactlyOnceGuarantee(boolean chained) throws Exception { + testRecoveryWithAssertion(DeliveryGuarantee.EXACTLY_ONCE, 1, chained); } - @Test - public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { - testRecoveryWithAssertion( - DeliveryGuarantee.EXACTLY_ONCE, - 2, - (records) -> - assertThat(records) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new))); + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints(boolean chained) + throws Exception { + testRecoveryWithAssertion(DeliveryGuarantee.EXACTLY_ONCE, 2, chained); } - @Test - public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception { + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testAbortTransactionsOfPendingCheckpointsAfterFailure( + boolean chained, + @TempDir File checkpointDir, + @InjectMiniCluster MiniCluster miniCluster, + @InjectClusterClient ClusterClient clusterClient) + throws Exception { // Run a first job failing during the async phase of a checkpoint to leave some // lingering transactions - final Configuration config = new Configuration(); + final Configuration config = createConfiguration(4); config.setString(StateBackendOptions.STATE_BACKEND, "filesystem"); - final File checkpointDir = temp.newFolder(); config.setString( CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); config.set( ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT, CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); config.set(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, 2); + JobID firstJobId = null; + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); try { - executeWithMapper(new FailAsyncCheckpointMapper(1), config, "firstPrefix"); + firstJobId = + executeWithMapper( + new FailAsyncCheckpointMapper(1), + checkpointedRecords, + config, + chained, + "firstPrefix", + clusterClient); } catch (Exception e) { - assertThat(e.getCause().getCause().getMessage()) - .contains("Exceeded checkpoint tolerable failure"); + assertThat(e).hasStackTraceContaining("Exceeded checkpoint tolerable failure"); } - final File completedCheckpoint = TestUtils.getMostRecentCompletedCheckpoint(checkpointDir); + final Optional completedCheckpoint = + CommonTestUtils.getLatestCompletedCheckpointPath(firstJobId, miniCluster); - config.set(SavepointConfigOptions.SAVEPOINT_PATH, completedCheckpoint.toURI().toString()); + assertThat(completedCheckpoint).isPresent(); + config.set(SavepointConfigOptions.SAVEPOINT_PATH, completedCheckpoint.get()); // Run a second job which aborts all lingering transactions and new consumer should // immediately see the newly written records - failed.get().set(true); + SharedReference failed = sharedObjects.add(new AtomicBoolean(true)); executeWithMapper( - new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, "newPrefix"); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); - assertThat(deserializeValues(collectedRecords)) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)); + new FailingCheckpointMapper(failed), + checkpointedRecords, + config, + chained, + "newPrefix", + clusterClient); + final List committedRecords = + deserializeValues(drainAllRecordsFromTopic(topic, true)); + assertThat(committedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); } - @Test - public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception { + @ParameterizedTest(name = "chained={0}") + @ValueSource(booleans = {true, false}) + public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint( + boolean chained, @InjectClusterClient ClusterClient clusterClient) throws Exception { // Run a first job opening 5 transactions one per subtask and fail in async checkpoint phase - final Configuration config = new Configuration(); - config.set(CoreOptions.DEFAULT_PARALLELISM, 5); try { - executeWithMapper(new FailAsyncCheckpointMapper(0), config, null); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); + Configuration config = createConfiguration(5); + executeWithMapper( + new FailAsyncCheckpointMapper(0), + checkpointedRecords, + config, + chained, + null, + clusterClient); } catch (Exception e) { assertThat(e.getCause().getCause().getMessage()) .contains("Exceeded checkpoint tolerable failure"); @@ -293,29 +314,46 @@ public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exce assertThat(deserializeValues(drainAllRecordsFromTopic(topic, true))).isEmpty(); // Second job aborts all transactions from previous runs with higher parallelism - config.set(CoreOptions.DEFAULT_PARALLELISM, 1); - failed.get().set(true); + SharedReference failed = sharedObjects.add(new AtomicBoolean(true)); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); + Configuration config = createConfiguration(1); executeWithMapper( - new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, null); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); - assertThat(deserializeValues(collectedRecords)) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)); + new FailingCheckpointMapper(failed), + checkpointedRecords, + config, + chained, + null, + clusterClient); + final List committedRecords = + deserializeValues(drainAllRecordsFromTopic(topic, true)); + assertThat(committedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); + } + + private static Configuration createConfiguration(int parallelism) { + final Configuration config = new Configuration(); + config.set(CoreOptions.DEFAULT_PARALLELISM, parallelism); + return config; } - private void executeWithMapper( + private JobID executeWithMapper( MapFunction mapper, + SharedReference> checkpointedRecords, Configuration config, - @Nullable String transactionalIdPrefix) + boolean chained, + @Nullable String transactionalIdPrefix, + @InjectClusterClient ClusterClient clusterClient) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(config); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); env.enableCheckpointing(100L); + if (!chained) { + env.disableOperatorChaining(); + } env.setRestartStrategy(RestartStrategies.noRestart()); final DataStreamSource source = env.fromSequence(1, 10); - final DataStream stream = source.map(mapper); + final DataStream stream = + source.map(mapper).map(new RecordFetcher(checkpointedRecords)).uid("fetcher"); final KafkaSinkBuilder builder = new KafkaSinkBuilder() .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) @@ -330,21 +368,33 @@ private void executeWithMapper( } builder.setTransactionalIdPrefix(transactionalIdPrefix); stream.sinkTo(builder.build()); - env.execute(); - checkProducerLeak(); + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + JobID jobID = clusterClient.submitJob(jobGraph).get(); + clusterClient.requestJobResult(jobID).get(); + return jobID; + } + + private void testRecoveryWithAssertion( + DeliveryGuarantee guarantee, int maxConcurrentCheckpoints) throws Exception { + testRecoveryWithAssertion(guarantee, maxConcurrentCheckpoints, true); } private void testRecoveryWithAssertion( - DeliveryGuarantee guarantee, - int maxConcurrentCheckpoints, - java.util.function.Consumer> recordsAssertion) + DeliveryGuarantee guarantee, int maxConcurrentCheckpoints, boolean chained) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(createConfiguration(1)); + if (!chained) { + env.disableOperatorChaining(); + } env.enableCheckpointing(300L); env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints); DataStreamSource source = env.fromSequence(1, 10); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); DataStream stream = - source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); + source.map(new FailingCheckpointMapper(sharedObjects.add(new AtomicBoolean(false)))) + .map(new RecordFetcher(checkpointedRecords)); stream.sinkTo( new KafkaSinkBuilder() @@ -359,42 +409,53 @@ private void testRecoveryWithAssertion( .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE); - recordsAssertion.accept(deserializeValues(collectedRecords)); - checkProducerLeak(); + List committedRecords = + deserializeValues( + drainAllRecordsFromTopic( + topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE)); + + if (guarantee == DeliveryGuarantee.AT_LEAST_ONCE) { + assertThat(committedRecords).containsAll(checkpointedRecords.get()); + } else if (guarantee == DeliveryGuarantee.EXACTLY_ONCE) { + assertThat(committedRecords) + .containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); + } + } + + private void writeRecordsToKafka(DeliveryGuarantee deliveryGuarantee) throws Exception { + writeRecordsToKafka(deliveryGuarantee, true); } - private void writeRecordsToKafka( - DeliveryGuarantee deliveryGuarantee, SharedReference expectedRecords) + private void writeRecordsToKafka(DeliveryGuarantee deliveryGuarantee, boolean chained) throws Exception { - final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(createConfiguration(1)); + if (!chained) { + env.disableOperatorChaining(); + } env.enableCheckpointing(100L); - final DataStream source = - env.addSource( - new InfiniteIntegerSource( - emittedRecordsCount, emittedRecordsWithCheckpoint)); - source.sinkTo( - new KafkaSinkBuilder() - .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) - .setDeliveryGuarantee(deliveryGuarantee) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setTopic(topic) - .setValueSerializationSchema(new RecordSerializer()) - .build()) - .setTransactionalIdPrefix("kafka-sink") - .build()); + final DataStream source = env.addSource(new InfiniteIntegerSource()); + SharedReference> checkpointedRecords = + sharedObjects.add(new ConcurrentSkipListSet<>()); + source.map(new RecordFetcher(checkpointedRecords)) + .sinkTo( + new KafkaSinkBuilder() + .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) + .setDeliveryGuarantee(deliveryGuarantee) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setTopic(topic) + .setValueSerializationSchema(new RecordSerializer()) + .build()) + .setTransactionalIdPrefix("kafka-sink") + .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic( - topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); - final long recordsCount = expectedRecords.get().get(); - assertThat(recordsCount).isEqualTo(collectedRecords.size()); - assertThat(deserializeValues(collectedRecords)) - .contains(LongStream.range(1, recordsCount + 1).boxed().toArray(Long[]::new)); - checkProducerLeak(); + final List collectedRecords = + deserializeValues( + drainAllRecordsFromTopic( + topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE)); + assertThat(collectedRecords).containsExactlyInAnyOrderElementsOf(checkpointedRecords.get()); } private static List deserializeValues(List> records) { @@ -423,7 +484,7 @@ private static Properties getKafkaClientConfiguration() { } private void createTestTopic(String topic, int numPartitions, short replicationFactor) - throws ExecutionException, InterruptedException, TimeoutException { + throws ExecutionException, InterruptedException { final CreateTopicsResult result = admin.createTopics( Collections.singletonList( @@ -431,8 +492,7 @@ private void createTestTopic(String topic, int numPartitions, short replicationF result.all().get(); } - private void deleteTestTopic(String topic) - throws ExecutionException, InterruptedException, TimeoutException { + private void deleteTestTopic(String topic) throws ExecutionException, InterruptedException { final DeleteTopicsResult result = admin.deleteTopics(Collections.singletonList(topic)); result.all().get(); } @@ -453,6 +513,63 @@ public byte[] serialize(Long element) { } } + /** + * Fetches records that have been successfully checkpointed. It relies on final checkpoints and + * subsumption to ultimately, emit all records that have been checkpointed. + * + *

      Note that the current implementation only works by operating on a set because on failure, + * we may up with duplicate records being added to the {@link #checkpointedRecords}. + * + *

      The fetcher uses three states to manage the records: + * + *

        + *
      1. {@link #recordsSinceLastCheckpoint} is used to buffer records between checkpoints. + *
      2. {@link #snapshottedRecords} is used to store the records that have been checkpointed. + *
      3. {@link #checkpointedRecords} is used to store snapshottedRecords where the checkpoint + * has been acknowledged. + *
      + * + *

      Records are promoted from data structure to the next (e.g. removed from the lower level). + */ + private static class RecordFetcher + implements MapFunction, CheckpointedFunction, CheckpointListener { + private final SharedReference> checkpointedRecords; + private final List recordsSinceLastCheckpoint = new ArrayList<>(); + private static final ListStateDescriptor STATE_DESCRIPTOR = + new ListStateDescriptor<>("committed-records", BasicTypeInfo.LONG_TYPE_INFO); + private ListState snapshottedRecords; + + private RecordFetcher(SharedReference> checkpointedRecords) { + this.checkpointedRecords = checkpointedRecords; + } + + @Override + public Long map(Long value) { + recordsSinceLastCheckpoint.add(value); + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // sync with shared object, this is guaranteed to sync eventually because of final + // checkpoint + checkpointedRecords.get().addAll(Lists.newArrayList(snapshottedRecords.get())); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + LOG.info( + "snapshotState {} @ {}", recordsSinceLastCheckpoint, context.getCheckpointId()); + snapshottedRecords.addAll(recordsSinceLastCheckpoint); + recordsSinceLastCheckpoint.clear(); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + snapshottedRecords = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); + } + } + private static class FailAsyncCheckpointMapper implements MapFunction, CheckpointedFunction { private static final ListStateDescriptor stateDescriptor = @@ -517,7 +634,7 @@ public int getLength() { } @Override - public void serialize(Integer record, DataOutputView target) throws IOException { + public void serialize(Integer record, DataOutputView target) { if (record != -1) { return; } @@ -525,17 +642,17 @@ public void serialize(Integer record, DataOutputView target) throws IOException } @Override - public Integer deserialize(DataInputView source) throws IOException { + public Integer deserialize(DataInputView source) { return 1; } @Override - public Integer deserialize(Integer reuse, DataInputView source) throws IOException { + public Integer deserialize(Integer reuse, DataInputView source) { return 1; } @Override - public void copy(DataInputView source, DataOutputView target) throws IOException {} + public void copy(DataInputView source, DataOutputView target) {} @Override public TypeSerializerSnapshot snapshotConfiguration() { @@ -552,84 +669,33 @@ public SlowSerializerSnapshot() { /** Fails after a checkpoint is taken and the next record was emitted. */ private static class FailingCheckpointMapper - implements MapFunction, CheckpointListener, CheckpointedFunction { + implements MapFunction, CheckpointListener { private final SharedReference failed; - private final SharedReference lastCheckpointedRecord; - - private volatile long lastSeenRecord; - private volatile long checkpointedRecord; - private volatile long lastCheckpointId = 0; - private final AtomicInteger emittedBetweenCheckpoint = new AtomicInteger(0); + private long lastCheckpointId = 0; + private int emittedBetweenCheckpoint = 0; - FailingCheckpointMapper( - SharedReference failed, - SharedReference lastCheckpointedRecord) { + FailingCheckpointMapper(SharedReference failed) { this.failed = failed; - this.lastCheckpointedRecord = lastCheckpointedRecord; } @Override public Long map(Long value) throws Exception { - lastSeenRecord = value; - if (lastCheckpointId >= 1 - && emittedBetweenCheckpoint.get() > 0 - && !failed.get().get()) { + if (lastCheckpointId >= 1 && emittedBetweenCheckpoint > 0 && !failed.get().get()) { failed.get().set(true); throw new RuntimeException("Planned exception."); } // Delay execution to ensure that at-least one checkpoint is triggered before finish Thread.sleep(50); - emittedBetweenCheckpoint.incrementAndGet(); + emittedBetweenCheckpoint++; return value; } @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - LOG.info("notifyCheckpointComplete {} @ {}", checkpointedRecord, checkpointId); + public void notifyCheckpointComplete(long checkpointId) { lastCheckpointId = checkpointId; - emittedBetweenCheckpoint.set(0); - lastCheckpointedRecord.get().set(checkpointedRecord); - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - LOG.info("snapshotState {} @ {}", lastSeenRecord, context.getCheckpointId()); - checkpointedRecord = lastSeenRecord; + emittedBetweenCheckpoint = 0; } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} - } - - private void checkProducerLeak() throws InterruptedException { - List> leaks = null; - for (int tries = 0; tries < 10; tries++) { - leaks = - Thread.getAllStackTraces().entrySet().stream() - .filter(this::findAliveKafkaThread) - .collect(Collectors.toList()); - if (leaks.isEmpty()) { - return; - } - Thread.sleep(1000); - } - - for (Map.Entry leak : leaks) { - leak.getKey().stop(); - } - fail( - "Detected producer leaks:\n" - + leaks.stream().map(this::format).collect(Collectors.joining("\n\n"))); - } - - private String format(Map.Entry leak) { - return leak.getKey().getName() + ":\n" + Joiner.on("\n").join(leak.getValue()); - } - - private boolean findAliveKafkaThread(Map.Entry threadStackTrace) { - return threadStackTrace.getKey().getState() != Thread.State.TERMINATED - && threadStackTrace.getKey().getName().contains("kafka-producer-network-thread"); } /** @@ -637,31 +703,23 @@ private boolean findAliveKafkaThread(Map.Entry thre * receiving the checkpoint completed event. */ private static final class InfiniteIntegerSource - implements SourceFunction, CheckpointListener, CheckpointedFunction { - - private final SharedReference emittedRecordsCount; - private final SharedReference emittedRecordsWithCheckpoint; + implements SourceFunction, CheckpointListener { private volatile boolean running = true; - private volatile long temp; - private Object lock; + private final AtomicInteger nextRecord = new AtomicInteger(); - InfiniteIntegerSource( - SharedReference emittedRecordsCount, - SharedReference emittedRecordsWithCheckpoint) { - this.emittedRecordsCount = emittedRecordsCount; - this.emittedRecordsWithCheckpoint = emittedRecordsWithCheckpoint; - } + InfiniteIntegerSource() {} @Override public void run(SourceContext ctx) throws Exception { - lock = ctx.getCheckpointLock(); + Object lock = ctx.getCheckpointLock(); while (running) { synchronized (lock) { - ctx.collect(emittedRecordsCount.get().addAndGet(1)); + ctx.collect((long) nextRecord.getAndIncrement()); Thread.sleep(1); } } + LOG.info("last emitted record {}", nextRecord.get() - 1); } @Override @@ -670,22 +728,9 @@ public void cancel() { } @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - emittedRecordsWithCheckpoint.get().set(temp); + public void notifyCheckpointComplete(long checkpointId) { running = false; LOG.info("notifyCheckpointCompleted {}", checkpointId); } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - temp = emittedRecordsCount.get().get(); - LOG.info( - "snapshotState, {}, {}", - context.getCheckpointId(), - emittedRecordsCount.get().get()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java new file mode 100644 index 000000000..84c1e4293 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkTest.java @@ -0,0 +1,194 @@ +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetFacetProvider; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.TypeDatasetFacetProvider; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamNode; +import org.apache.flink.streaming.api.lineage.LineageVertex; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Optional; +import java.util.Properties; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link KafkaSink}. */ +public class KafkaSinkTest { + + Properties kafkaProperties; + + @BeforeEach + void setup() { + kafkaProperties = new Properties(); + kafkaProperties.put("bootstrap.servers", "host1;host2"); + } + + @Test + public void testGetLineageVertexWhenSerializerNotAnKafkaDatasetFacetProvider() { + KafkaRecordSerializationSchema recordSerializer = + new KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider(); + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); + + assertThat(sink.getLineageVertex().datasets()).isEmpty(); + } + + @Test + public void testGetLineageVertexWhenNoKafkaDatasetFacetReturnedFromSerializer() { + KafkaRecordSerializationSchema recordSerializer = + new KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider(); + + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, new Properties(), "", recordSerializer); + + assertThat(sink.getLineageVertex().datasets()).isEmpty(); + } + + @Test + public void testGetLineageVertex() { + KafkaRecordSerializationSchema recordSerializer = + new TestingKafkaRecordSerializationSchema(); + + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, kafkaProperties, "", recordSerializer); + + LineageVertex lineageVertex = sink.getLineageVertex(); + + assertThat(lineageVertex.datasets().get(0).namespace()).isEqualTo("kafka://host1"); + assertThat(lineageVertex.datasets().get(0).name()).isEqualTo("topic1"); + + assertThat( + lineageVertex + .datasets() + .get(0) + .facets() + .get(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME)) + .hasFieldOrPropertyWithValue("properties", kafkaProperties) + .hasFieldOrPropertyWithValue( + "topicIdentifier", + DefaultKafkaDatasetIdentifier.ofTopics( + Collections.singletonList("topic1"))); + + assertThat( + lineageVertex + .datasets() + .get(0) + .facets() + .get(DefaultTypeDatasetFacet.TYPE_FACET_NAME)) + .hasFieldOrPropertyWithValue("typeInformation", TypeInformation.of(String.class)); + } + + @Test + public void testCoLocation() { + String colocationKey = "testCoLocation"; + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, + kafkaProperties, + colocationKey, + new TestingKafkaRecordSerializationSchema()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.fromData(1).sinkTo(sink); + + StreamGraph streamGraph = env.getStreamGraph(); + assertThat(streamGraph.getStreamNodes()) + .filteredOn(node -> !node.getInEdges().isEmpty()) + .hasSize(2) // writer and committer + .extracting(StreamNode::getCoLocationGroup) + .containsOnly(colocationKey); + } + + @Test + public void testPreserveCustomCoLocation() { + String colocationKey = "testPreserveCustomCoLocation"; + String customColocationKey = "customCoLocation"; + KafkaSink sink = + new KafkaSink<>( + DeliveryGuarantee.EXACTLY_ONCE, + kafkaProperties, + colocationKey, + new TestingKafkaRecordSerializationSchema()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSink stream = env.fromData(1).sinkTo(sink); + stream.getTransformation().setCoLocationGroupKey(customColocationKey); + + StreamGraph streamGraph = env.getStreamGraph(); + assertThat(streamGraph.getStreamNodes()) + .filteredOn(node -> !node.getInEdges().isEmpty()) + .hasSize(2) // writer and committer + .extracting(StreamNode::getCoLocationGroup) + .containsOnly(customColocationKey); + } + + private static class KafkaRecordSerializationSchemaWithoutKafkaDatasetProvider + implements KafkaRecordSerializationSchema { + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + } + + private static class KafkaRecordSerializationSchemaWithEmptyKafkaDatasetProvider + implements KafkaRecordSerializationSchema, KafkaDatasetFacetProvider { + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + + @Override + public Optional getKafkaDatasetFacet() { + return Optional.empty(); + } + } + + private static class TestingKafkaRecordSerializationSchema + implements KafkaRecordSerializationSchema, + KafkaDatasetFacetProvider, + TypeDatasetFacetProvider { + + @Override + public Optional getKafkaDatasetFacet() { + return Optional.of( + new DefaultKafkaDatasetFacet( + DefaultKafkaDatasetIdentifier.ofTopics( + Collections.singletonList("topic1")))); + } + + @Nullable + @Override + public ProducerRecord serialize( + Object element, KafkaSinkContext context, Long timestamp) { + return null; + } + + @Override + public Optional getTypeDatasetFacet() { + return Optional.of(new DefaultTypeDatasetFacet(TypeInformation.of(String.class))); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java index 84d732dbf..aca541455 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java @@ -29,8 +29,6 @@ import org.junit.After; import org.junit.ClassRule; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import java.util.ArrayList; @@ -44,26 +42,26 @@ import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.Ongoing; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareAbort; import static org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionState.PrepareCommit; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaTransactionLog} to retrieve abortable Kafka transactions. */ public class KafkaTransactionLogITCase extends TestLogger { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); private static final String TOPIC_NAME = "kafkaTransactionLogTest"; private static final String TRANSACTIONAL_ID_PREFIX = "kafka-log"; @ClassRule public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); + createKafkaContainer(KafkaTransactionLogITCase.class).withEmbeddedZookeeper(); private final List> openProducers = new ArrayList<>(); @After public void tearDown() { openProducers.forEach(Producer::close); + checkProducerLeak(); } @Test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java new file mode 100644 index 000000000..9ca558f33 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterFaultToleranceITCase.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.TimeoutException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.Properties; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; + +/** Tests for the standalone KafkaWriter in case of fault tolerance. */ +@ExtendWith(TestLoggerExtension.class) +public class KafkaWriterFaultToleranceITCase extends KafkaWriterTestBase { + private static final String INIT_KAFKA_RETRIES = "0"; + private static final String INIT_KAFKA_REQUEST_TIMEOUT_MS = "1000"; + private static final String INIT_KAFKA_MAX_BLOCK_MS = "1000"; + private static final String INIT_KAFKA_DELIVERY_TIMEOUT_MS = "1000"; + + @Test + void testWriteExceptionWhenKafkaUnavailable() throws Exception { + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + try (KafkaWriter writer = + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null))) { + + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + + try { + writer.getCurrentProducer().flush(); + assertThatCode(() -> writer.write(1, SINK_WRITER_CONTEXT)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + } + + @Test + void testFlushExceptionWhenKafkaUnavailable() throws Exception { + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + try (KafkaWriter writer = + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null))) { + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + try { + assertThatCode(() -> writer.flush(false)) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + } + + @Test + void testCloseExceptionWhenKafkaUnavailable() throws Exception { + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + + KafkaWriter writer = + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null)); + + writer.write(1, SINK_WRITER_CONTEXT); + + KAFKA_CONTAINER.stop(); + + try { + writer.getCurrentProducer().flush(); + // closing producer resource throws exception first + assertThatCode(() -> writer.close()) + .hasRootCauseExactlyInstanceOf(NetworkException.class); + } catch (Exception e) { + writer.close(); + throw e; + } finally { + KAFKA_CONTAINER.start(); + } + } + + @Test + void testMailboxExceptionWhenKafkaUnavailable() throws Exception { + SinkInitContext sinkInitContext = + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + + try (KafkaWriter writer = + createWriter(DeliveryGuarantee.AT_LEAST_ONCE, sinkInitContext)) { + + KAFKA_CONTAINER.stop(); + + writer.write(1, SINK_WRITER_CONTEXT); + + try { + writer.getCurrentProducer().flush(); + + assertThatCode( + () -> { + while (sinkInitContext.getMailboxExecutor().tryYield()) { + // execute all mails + } + }) + .hasRootCauseExactlyInstanceOf(TimeoutException.class); + } finally { + KAFKA_CONTAINER.start(); + } + } + } + + @Override + protected Properties getKafkaClientConfiguration() { + Properties properties = super.getKafkaClientConfiguration(); + + // reduce the default vault for test case + properties.setProperty("retries", INIT_KAFKA_RETRIES); + properties.setProperty("request.timeout.ms", INIT_KAFKA_REQUEST_TIMEOUT_MS); + properties.setProperty("max.block.ms", INIT_KAFKA_MAX_BLOCK_MS); + properties.setProperty("delivery.timeout.ms", INIT_KAFKA_DELIVERY_TIMEOUT_MS); + + return properties; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 98f64ed13..805690afa 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -17,107 +17,35 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.common.operators.ProcessingTimeService; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.connector.sink2.Sink; -import org.apache.flink.api.connector.sink2.SinkWriter; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.internal.FlinkKafkaInternalProducer; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; -import org.apache.flink.metrics.testutils.MetricListener; -import org.apache.flink.runtime.mailbox.SyncMailboxExecutor; -import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.util.UserCodeClassLoader; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.Network; - -import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Optional; -import java.util.OptionalLong; -import java.util.PriorityQueue; import java.util.Properties; -import java.util.concurrent.ScheduledFuture; -import java.util.function.Consumer; import java.util.stream.IntStream; -import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the standalone KafkaWriter. */ @ExtendWith(TestLoggerExtension.class) -public class KafkaWriterITCase { - - private static final Logger LOG = LoggerFactory.getLogger(KafkaWriterITCase.class); - private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; - private static final Network NETWORK = Network.newNetwork(); - private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; - private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - private String topic; - - private MetricListener metricListener; - private TriggerTimeService timeService; - - private static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); - - @BeforeAll - public static void beforeAll() { - KAFKA_CONTAINER.start(); - } - - @AfterAll - public static void afterAll() { - KAFKA_CONTAINER.stop(); - } - - @BeforeEach - public void setUp(TestInfo testInfo) { - metricListener = new MetricListener(); - timeService = new TriggerTimeService(); - topic = testInfo.getDisplayName().replaceAll("\\W", ""); - } - +public class KafkaWriterITCase extends KafkaWriterTestBase { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter ignored = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + try (final KafkaWriter ignored = createWriter(guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME).isPresent()).isTrue(); } } @@ -131,50 +59,49 @@ public void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception @Test public void testIncreasingRecordBasedCounters() throws Exception { - final OperatorIOMetricGroup operatorIOMetricGroup = - UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock( - metricListener.getMetricGroup(), operatorIOMetricGroup); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { + createWriter( + DeliveryGuarantee.AT_LEAST_ONCE, + new SinkInitContext(metricGroup, timeService, null))) { final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); final Counter numRecordsSendErrors = metricGroup.getNumRecordsSendErrorsCounter(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); + + // ApiVersionsRequest etc. is sent on initialization, so establish some baseline + writer.write(0, SINK_WRITER_CONTEXT); + writer.flush(false); // ensure data is actually written + timeService.trigger(); // sync byte count + long baselineCount = numBytesOut.getCount(); + assertThat(numRecordsOut.getCount()).isEqualTo(1); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); // elements for which the serializer returns null should be silently skipped writer.write(null, SINK_WRITER_CONTEXT); - timeService.trigger(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); + writer.flush(false); + timeService.trigger(); // sync byte count + assertThat(numBytesOut.getCount()).isEqualTo(baselineCount); + assertThat(numRecordsOut.getCount()).isEqualTo(1); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); // but elements for which a non-null producer record is returned should count writer.write(1, SINK_WRITER_CONTEXT); - timeService.trigger(); - assertThat(numRecordsOut.getCount()).isEqualTo(1); + writer.flush(false); + timeService.trigger(); // sync byte count + assertThat(numBytesOut.getCount()).isGreaterThan(baselineCount); + assertThat(numRecordsOut.getCount()).isEqualTo(2); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); - assertThat(numBytesOut.getCount()).isGreaterThan(0L); } } @Test public void testCurrentSendTimeMetric() throws Exception { - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), - DeliveryGuarantee.AT_LEAST_ONCE, - metricGroup)) { + try (final KafkaWriter writer = createWriter(DeliveryGuarantee.AT_LEAST_ONCE)) { final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); assertThat(currentSendTime.isPresent()).isTrue(); @@ -193,98 +120,31 @@ public void testCurrentSendTimeMetric() throws Exception { } }); assertThat(currentSendTime.get().getValue()).isGreaterThan(0L); - } - } - - @Test - void testNumRecordsOutErrorsCounterMetric() throws Exception { - Properties properties = getKafkaClientConfiguration(); - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); - - try (final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup)) { - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - writer.write(1, SINK_WRITER_CONTEXT); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); - - final String transactionalId = writer.getCurrentProducer().getTransactionalId(); - - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { - - producer.initTransactions(); - producer.beginTransaction(); - producer.send(new ProducerRecord(topic, "2".getBytes())); - producer.commitTransaction(); - } - - writer.write(3, SINK_WRITER_CONTEXT); - writer.flush(false); - writer.prepareCommit(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(1L); + writer.flush(true); } } @Test public void testMetadataPublisher() throws Exception { List metadataList = new ArrayList<>(); + SinkWriterMetricGroup sinkWriterMetricGroup = createSinkWriterMetricGroup(); try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), + createWriter( DeliveryGuarantee.AT_LEAST_ONCE, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - meta -> metadataList.add(meta.toString()))) { + new SinkInitContext( + sinkWriterMetricGroup, + timeService, + meta -> metadataList.add(meta.topic() + "@" + meta.offset())))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { writer.write(1, SINK_WRITER_CONTEXT); - expected.add("testMetadataPublisher-0@" + i); + expected.add("testMetadataPublisher@" + i); } writer.flush(false); - assertThat(metadataList).usingRecursiveComparison().isEqualTo(expected); + assertThat(metadataList).containsExactlyInAnyOrderElementsOf(expected); } } - /** Test that producer is not accidentally recreated or pool is used. */ - @Test - void testLingeringTransaction() throws Exception { - final KafkaWriter failedWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); - - // create two lingering transactions - failedWriter.flush(false); - failedWriter.prepareCommit(); - failedWriter.snapshotState(1); - failedWriter.flush(false); - failedWriter.prepareCommit(); - failedWriter.snapshotState(2); - - try (final KafkaWriter recoveredWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { - recoveredWriter.write(1, SINK_WRITER_CONTEXT); - - recoveredWriter.flush(false); - Collection committables = recoveredWriter.prepareCommit(); - recoveredWriter.snapshotState(1); - assertThat(committables).hasSize(1); - final KafkaCommittable committable = committables.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); - - committable.getProducer().get().getObject().commitTransaction(); - - List> records = - drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); - assertThat(records).hasSize(1); - } - - failedWriter.close(); - } - /** Test that producer is not accidentally recreated or pool is used. */ @ParameterizedTest @EnumSource( @@ -292,10 +152,7 @@ void testLingeringTransaction() throws Exception { names = "EXACTLY_ONCE", mode = EnumSource.Mode.EXCLUDE) void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { - assertThat(writer.getProducerPool()).hasSize(0); - + try (final KafkaWriter writer = createWriter(guarantee)) { FlinkKafkaInternalProducer firstProducer = writer.getCurrentProducer(); writer.flush(false); Collection committables = writer.prepareCommit(); @@ -305,81 +162,6 @@ void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exce assertThat(writer.getCurrentProducer() == firstProducer) .as("Expected same producer") .isTrue(); - assertThat(writer.getProducerPool()).hasSize(0); - } - } - - /** Test that producers are reused when committed. */ - @Test - void usePoolForTransactional() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); - - writer.flush(false); - Collection committables0 = writer.prepareCommit(); - writer.snapshotState(1); - assertThat(committables0).hasSize(1); - final KafkaCommittable committable = committables0.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); - - FlinkKafkaInternalProducer firstProducer = - committable.getProducer().get().getObject(); - assertThat(firstProducer != writer.getCurrentProducer()) - .as("Expected different producer") - .isTrue(); - - // recycle first producer, KafkaCommitter would commit it and then return it - assertThat(writer.getProducerPool()).hasSize(0); - firstProducer.commitTransaction(); - committable.getProducer().get().close(); - assertThat(writer.getProducerPool()).hasSize(1); - - writer.flush(false); - Collection committables1 = writer.prepareCommit(); - writer.snapshotState(2); - assertThat(committables1).hasSize(1); - final KafkaCommittable committable1 = committables1.stream().findFirst().get(); - assertThat(committable1.getProducer().isPresent()).isTrue(); - - assertThat(firstProducer == writer.getCurrentProducer()) - .as("Expected recycled producer") - .isTrue(); - } - } - - /** - * Tests that open transactions are automatically aborted on close such that successive writes - * succeed. - */ - @Test - void testAbortOnClose() throws Exception { - Properties properties = getKafkaClientConfiguration(); - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { - writer.write(1, SINK_WRITER_CONTEXT); - assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); - } - - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { - writer.write(2, SINK_WRITER_CONTEXT); - writer.flush(false); - Collection committables = writer.prepareCommit(); - writer.snapshotState(1L); - - // manually commit here, which would only succeed if the first transaction was aborted - assertThat(committables).hasSize(1); - final KafkaCommittable committable = committables.stream().findFirst().get(); - String transactionalId = committable.getTransactionalId(); - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { - producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); - producer.commitTransaction(); - } - - assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(1); } } @@ -388,180 +170,12 @@ private void assertKafkaMetricNotPresent( final Properties config = getKafkaClientConfiguration(); config.put(configKey, configValue); try (final KafkaWriter ignored = - createWriterWithConfiguration(config, guarantee)) { + createWriter( + builder -> + builder.setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee), + createInitContext())) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent(); } } - - private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) { - return createWriterWithConfiguration( - config, - guarantee, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); - } - - private KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) { - return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); - } - - private KafkaWriter createWriterWithConfiguration( - Properties config, - DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) { - return new KafkaWriter<>( - guarantee, - config, - "test-prefix", - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer), - new DummyRecordSerializer(), - new DummySchemaContext(), - ImmutableList.of()); - } - - private static Properties getKafkaClientConfiguration() { - final Properties standardProps = new Properties(); - standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); - standardProps.put("group.id", "kafkaWriter-tests"); - standardProps.put("enable.auto.commit", false); - standardProps.put("key.serializer", ByteArraySerializer.class.getName()); - standardProps.put("value.serializer", ByteArraySerializer.class.getName()); - standardProps.put("auto.offset.reset", "earliest"); - return standardProps; - } - - private static class SinkInitContext implements Sink.InitContext { - - private final SinkWriterMetricGroup metricGroup; - private final ProcessingTimeService timeService; - @Nullable private final Consumer metadataConsumer; - - SinkInitContext( - SinkWriterMetricGroup metricGroup, - ProcessingTimeService timeService, - @Nullable Consumer metadataConsumer) { - this.metricGroup = metricGroup; - this.timeService = timeService; - this.metadataConsumer = metadataConsumer; - } - - @Override - public UserCodeClassLoader getUserCodeClassLoader() { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public MailboxExecutor getMailboxExecutor() { - return new SyncMailboxExecutor(); - } - - @Override - public ProcessingTimeService getProcessingTimeService() { - return timeService; - } - - @Override - public int getSubtaskId() { - return 0; - } - - @Override - public int getNumberOfParallelSubtasks() { - return 1; - } - - @Override - public int getAttemptNumber() { - return 0; - } - - @Override - public SinkWriterMetricGroup metricGroup() { - return metricGroup; - } - - @Override - public OptionalLong getRestoredCheckpointId() { - return OptionalLong.empty(); - } - - @Override - public SerializationSchema.InitializationContext - asSerializationSchemaInitializationContext() { - return null; - } - - @Override - public Optional> metadataConsumer() { - return Optional.ofNullable((Consumer) metadataConsumer); - } - } - - private class DummyRecordSerializer implements KafkaRecordSerializationSchema { - @Override - public ProducerRecord serialize( - Integer element, KafkaSinkContext context, Long timestamp) { - if (element == null) { - // in general, serializers should be allowed to skip invalid elements - return null; - } - return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array()); - } - } - - private static class DummySchemaContext implements SerializationSchema.InitializationContext { - - @Override - public MetricGroup getMetricGroup() { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public UserCodeClassLoader getUserCodeClassLoader() { - throw new UnsupportedOperationException("Not implemented."); - } - } - - private static class DummySinkWriterContext implements SinkWriter.Context { - @Override - public long currentWatermark() { - return 0; - } - - @Override - public Long timestamp() { - return null; - } - } - - private static class TriggerTimeService implements ProcessingTimeService { - - private final PriorityQueue> registeredCallbacks = - new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); - - @Override - public long getCurrentProcessingTime() { - return 0; - } - - @Override - public ScheduledFuture registerTimer( - long time, ProcessingTimeCallback processingTimerCallback) { - registeredCallbacks.add(new Tuple2<>(time, processingTimerCallback)); - return null; - } - - public void trigger() throws Exception { - final Tuple2 registered = registeredCallbacks.poll(); - if (registered == null) { - LOG.warn("Triggered time service but no callback was registered."); - return; - } - registered.f1.onProcessingTime(registered.f0); - } - } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java new file mode 100644 index 000000000..4f492966c --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterTestBase.java @@ -0,0 +1,306 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.sink; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; +import org.apache.flink.connector.kafka.sink.internal.BackchannelFactory; +import org.apache.flink.connector.kafka.sink.internal.WritableBackchannel; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.util.UserCodeClassLoader; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.PriorityQueue; +import java.util.Properties; +import java.util.concurrent.ScheduledFuture; +import java.util.function.Consumer; + +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; +import static org.apache.kafka.clients.admin.AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG; + +/** Test base for KafkaWriter. */ +@Testcontainers +public abstract class KafkaWriterTestBase { + + protected static final Logger LOG = LoggerFactory.getLogger(KafkaWriterTestBase.class); + protected static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; + protected static final Network NETWORK = Network.newNetwork(); + protected static final String KAFKA_METRIC_WITH_GROUP_NAME = + "KafkaProducer.incoming-byte-total"; + protected static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); + private static final String TEST_PREFIX = "test-prefix"; + private int writerIndex; + private static final int SUB_ID = 0; + private static final int ATTEMPT = 0; + protected static String topic; + + protected MetricListener metricListener; + protected TriggerTimeService timeService; + + @Container + public static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(KafkaWriterTestBase.class) + .withEmbeddedZookeeper() + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + + @BeforeEach + public void setUp(TestInfo testInfo) { + metricListener = new MetricListener(); + timeService = new TriggerTimeService(); + topic = testInfo.getDisplayName().replaceAll("\\W", ""); + Map properties = new java.util.HashMap<>(); + properties.put(BOOTSTRAP_SERVERS_CONFIG, KAFKA_CONTAINER.getBootstrapServers()); + try (Admin admin = AdminClient.create(properties)) { + admin.createTopics(Collections.singleton(new NewTopic(topic, 10, (short) 1))); + } + } + + @AfterEach + public void teardown() { + checkProducerLeak(); + } + + > T createWriter(DeliveryGuarantee guarantee) throws IOException { + return createWriter(guarantee, createInitContext()); + } + + > T createWriter( + DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) throws IOException { + return createWriter(builder -> builder.setDeliveryGuarantee(guarantee), sinkInitContext); + } + + @SuppressWarnings("unchecked") + > T createWriter( + Consumer> sinkBuilderAdjuster, SinkInitContext sinkInitContext) + throws IOException { + return (T) createSink(sinkBuilderAdjuster).createWriter(sinkInitContext); + } + + KafkaSink createSink(Consumer> sinkBuilderAdjuster) { + KafkaSinkBuilder builder = + KafkaSink.builder() + .setKafkaProducerConfig(getKafkaClientConfiguration()) + .setTransactionalIdPrefix(TEST_PREFIX + writerIndex++) + .setRecordSerializer(new DummyRecordSerializer()); + sinkBuilderAdjuster.accept(builder); + return builder.build(); + } + + SinkInitContext createInitContext() { + return new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + } + + WritableBackchannel getBackchannel(ExactlyOnceKafkaWriter writer) { + return BackchannelFactory.getInstance() + .getWritableBackchannel(SUB_ID, ATTEMPT, writer.getTransactionalIdPrefix()); + } + + protected SinkWriterMetricGroup createSinkWriterMetricGroup() { + DummyOperatorMetricGroup operatorMetricGroup = + new DummyOperatorMetricGroup(metricListener.getMetricGroup()); + return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); + } + + protected Properties getKafkaClientConfiguration() { + final Properties standardProps = new Properties(); + standardProps.put("bootstrap.servers", KAFKA_CONTAINER.getBootstrapServers()); + standardProps.put("group.id", "kafkaWriter-tests"); + standardProps.put("enable.auto.commit", false); + standardProps.put("key.serializer", ByteArraySerializer.class.getName()); + standardProps.put("value.serializer", ByteArraySerializer.class.getName()); + standardProps.put("auto.offset.reset", "earliest"); + return standardProps; + } + + /** mock sink context for initializing KafkaWriter. */ + protected static class SinkInitContext extends TestSinkInitContext { + + protected final SinkWriterMetricGroup metricGroup; + protected final ProcessingTimeService timeService; + @Nullable protected final Consumer metadataConsumer; + + SinkInitContext( + SinkWriterMetricGroup metricGroup, + ProcessingTimeService timeService, + @Nullable Consumer metadataConsumer) { + this.metricGroup = metricGroup; + this.timeService = timeService; + this.metadataConsumer = metadataConsumer; + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + throw new UnsupportedOperationException("Not implemented."); + } + + @Override + public ProcessingTimeService getProcessingTimeService() { + return timeService; + } + + @Override + public int getSubtaskId() { + return SUB_ID; + } + + @Override + public int getNumberOfParallelSubtasks() { + return 1; + } + + @Override + public int getAttemptNumber() { + return ATTEMPT; + } + + @Override + public SinkWriterMetricGroup metricGroup() { + return metricGroup; + } + + @Override + public OptionalLong getRestoredCheckpointId() { + return OptionalLong.empty(); + } + + @Override + public SerializationSchema.InitializationContext + asSerializationSchemaInitializationContext() { + return null; + } + + @Override + public Optional> metadataConsumer() { + return Optional.ofNullable((Consumer) metadataConsumer); + } + } + + /** mock recordSerializer for KafkaSink. */ + protected static class DummyRecordSerializer + implements KafkaRecordSerializationSchema { + @Override + public ProducerRecord serialize( + Integer element, KafkaSinkContext context, Long timestamp) { + if (element == null) { + // in general, serializers should be allowed to skip invalid elements + return null; + } + return new ProducerRecord<>(topic, ByteBuffer.allocate(4).putInt(element).array()); + } + } + + /** + * mock context for KafkaWriter#write(java.lang.Object, + * org.apache.flink.api.connector.sink2.SinkWriter.Context). + */ + protected static class DummySinkWriterContext implements SinkWriter.Context { + @Override + public long currentWatermark() { + return ATTEMPT; + } + + @Override + public Long timestamp() { + return null; + } + } + + /** mock metrics group for initializing KafkaWriter. */ + protected static class DummyOperatorMetricGroup extends ProxyMetricGroup + implements OperatorMetricGroup { + + private final OperatorIOMetricGroup operatorIOMetricGroup; + + public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { + super(parentMetricGroup); + this.operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() + .getIOMetricGroup(); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return operatorIOMetricGroup; + } + } + + /** mock time service for KafkaWriter. */ + protected static class TriggerTimeService implements ProcessingTimeService { + + private final PriorityQueue> registeredCallbacks = + new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); + + @Override + public long getCurrentProcessingTime() { + return ATTEMPT; + } + + @Override + public ScheduledFuture registerTimer( + long time, ProcessingTimeCallback processingTimerCallback) { + registeredCallbacks.add(new Tuple2<>(time, processingTimerCallback)); + return null; + } + + public void trigger() throws Exception { + final Tuple2 registered = registeredCallbacks.poll(); + if (registered == null) { + LOG.warn("Triggered time service but no callback was registered."); + return; + } + registered.f1.onProcessingTime(registered.f0); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java index 982d58e33..73adc42c2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java @@ -17,6 +17,7 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.connector.kafka.sink.internal.TransactionalIdFactory; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java deleted file mode 100644 index 1b65c9d33..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -package org.apache.flink.connector.kafka.sink; - -import org.apache.flink.util.TestLogger; - -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; - -import org.junit.Test; - -import java.util.List; -import java.util.Map; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Tests for {@link TransactionsToAbortChecker}. */ -public class TransactionToAbortCheckerTest extends TestLogger { - - public static final String ABORT = "abort"; - - @Test - public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() { - final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L, 2, 3L), 0); - - // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = - ImmutableMap.of( - 0, ImmutableMap.of(2L, ABORT, 1L, ABORT), - 2, ImmutableMap.of(3L, ABORT, 4L, ABORT), - 3, ImmutableMap.of(3L, "keep", 4L, "keep")); - - final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); - assertThat(transactionsToAbort).hasSize(4); - assertThatAbortCorrectTransaction(transactionsToAbort); - } - - @Test - public void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() { - final TransactionsToAbortChecker checker = - new TransactionsToAbortChecker(2, ImmutableMap.of(0, 1L), 0); - - // abort recovered subtasksId with equal or higher checkpoint offset - final Map> openTransactions = - ImmutableMap.of( - 0, ImmutableMap.of(2L, ABORT, 1L, ABORT), - 2, ImmutableMap.of(1L, ABORT), - 3, ImmutableMap.of(1L, "keep"), - 4, ImmutableMap.of(1L, ABORT), - 5, ImmutableMap.of(1L, "keep")); - - final List transactionsToAbort = checker.getTransactionsToAbort(openTransactions); - assertThat(transactionsToAbort).hasSize(4); - assertThatAbortCorrectTransaction(transactionsToAbort); - } - - private static void assertThatAbortCorrectTransaction(List abortedTransactions) { - assertThat(abortedTransactions.stream().allMatch(t -> t.equals(ABORT))).isTrue(); - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java new file mode 100644 index 000000000..1fd64935d --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/BackchannelImplTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.flink.util.function.RunnableWithException; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinTask; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +class BackchannelImplTest { + + private static final String PREFIX = "PREFIX"; + private static final int ATTEMPT = 0; + private static final BackchannelFactory FACTORY = BackchannelFactory.getInstance(); + + @Test + public void testBasicSend() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX); + ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + String message = "Test message"; + + writable.send(message); + + assertThat(readable.poll()).isEqualTo(message); + } + } + + @Test + public void testSendBeforeEstablish() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(writable.isEstablished()).isFalse(); + + String message = "Test message"; + + writable.send(message); + + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + assertThat(readable.poll()).isEqualTo(message); + } + } + } + + @Test + public void testPollBeforeEstablish() { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isFalse(); + assertThat(readable.poll()).isNull(); + } + } + + @Test + public void testBasicSendTwoThreads() throws InterruptedException, BrokenBarrierException { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeSend = new CyclicBarrier(2); + CyclicBarrier afterSend = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + beforeSend.await(); + afterSend.await(); + assertThat(readable.poll()).isEqualTo(message); + } + }); + + beforeSend.await(); + writable.send(message); + afterSend.await(); + task.join(); + } + } + + @Test + public void testSendBeforeEstablishTwoThreads() + throws BrokenBarrierException, InterruptedException { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeEstablish = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + beforeEstablish.await(); + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.poll()).isEqualTo(message); + } + }); + + writable.send(message); + beforeEstablish.await(); + task.join(); + } + } + + @Test + public void testPollBeforeEstablishTwoThreads() + throws BrokenBarrierException, InterruptedException { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + CyclicBarrier beforeEstablish = new CyclicBarrier(2); + CyclicBarrier afterEstablish = new CyclicBarrier(2); + CyclicBarrier afterSend = new CyclicBarrier(2); + + String message = "Test message"; + ForkJoinTask task = + runInParallel( + () -> { + beforeEstablish.await(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + afterEstablish.await(); + writable.send(message); + afterSend.await(); + } + }); + + try { + assertThat(readable.isEstablished()).isFalse(); + assertThat(readable.poll()).isNull(); + beforeEstablish.await(); + afterEstablish.await(); + assertThat(readable.isEstablished()).isTrue(); + afterSend.await(); + assertThat(readable.poll()).isEqualTo(message); + } finally { + // make sure to join first before exiting the test or else cleanup did not properly + // happen + task.join(); + // writable channel cleaned up + assertThat(readable.isEstablished()).isFalse(); + } + } + } + + @Test + void testDuplicatePrefix() { + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThatCode(() -> FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) + .hasMessageContaining("duplicate"); + } + } + + @Test + void testPrefixReuse() { + try (ReadableBackchannel readable = + FACTORY.getReadableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isFalse(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + } + assertThat(readable.isEstablished()).isFalse(); + try (WritableBackchannel writable = + FACTORY.getWritableBackchannel(1, ATTEMPT, PREFIX)) { + assertThat(readable.isEstablished()).isTrue(); + } + assertThat(readable.isEstablished()).isFalse(); + } + } + + private static ForkJoinTask runInParallel(RunnableWithException r) { + // convert to callable to allow exceptions + return ForkJoinPool.commonPool() + .submit( + () -> { + r.run(); + return true; + }); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java new file mode 100644 index 000000000..a831b8afe --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/internal/ProducerPoolImplITCase.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.sink.internal; + +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.util.Properties; +import java.util.function.Consumer; + +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; +import static org.assertj.core.api.Assertions.assertThat; + +@Testcontainers +class ProducerPoolImplITCase { + + public static final Consumer> INIT = p -> {}; + public static final String TRANSACTIONAL_ID = "test-transactional-id"; + + @Container + public static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(ProducerPoolImplITCase.class).withEmbeddedZookeeper(); + + @AfterEach + void checkLeak() { + checkProducerLeak(); + } + + @Test + void testGetTransactionalProducer() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(producer).isNotNull(); + assertThat(producer.getTransactionalId()).isEqualTo(TRANSACTIONAL_ID); + assertThat(producer.isInTransaction()).isFalse(); + // everything prepared to being the transaction + producer.beginTransaction(); + // no explicit closing of producer - pool should also clean up the producer + } + } + + /** Tests direct recycling as used during abortion of transactions. */ + @Test + void testRecycleProducer() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + + assertThat(producerPool.getProducers()).isEmpty(); + producerPool.recycle(producer); + assertThat(producerPool.getProducers()).contains(producer); + + FlinkKafkaInternalProducer newProducer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(newProducer).isSameAs(producer); + } + } + + /** Tests indirect recycling triggered through the backchannel. */ + @Test + void testRecycleByTransactionId() throws Exception { + try (ProducerPoolImpl producerPool = new ProducerPoolImpl(getProducerConfig(), INIT)) { + FlinkKafkaInternalProducer producer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + + assertThat(producerPool.getProducers()).isEmpty(); + producer.beginTransaction(); + producerPool.recycleByTransactionId(TRANSACTIONAL_ID); + assertThat(producerPool.getProducers()).contains(producer); + // forcefully reset transaction state for split brain scenarios + assertThat(producer.isInTransaction()).isFalse(); + + FlinkKafkaInternalProducer newProducer = + producerPool.getTransactionalProducer(TRANSACTIONAL_ID, 1L); + assertThat(newProducer).isSameAs(producer); + } + } + + private static Properties getProducerConfig() { + Properties kafkaProducerConfig = new Properties(); + kafkaProducerConfig.put( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, KAFKA_CONTAINER.getBootstrapServers()); + kafkaProducerConfig.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProducerConfig.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + return kafkaProducerConfig; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java index 236e9618f..ca777bc73 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java @@ -27,15 +27,20 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -191,6 +196,50 @@ public void testSettingCustomKafkaSubscriber() { "Cannot use partitions for consumption because a ExampleCustomSubscriber is already set for consumption."); } + @ParameterizedTest + @MethodSource("provideSettingCustomDeserializerTestParameters") + public void testSettingCustomDeserializer(String propertyKey, String propertyValue) { + final KafkaSource kafkaSource = + getBasicBuilder().setProperty(propertyKey, propertyValue).build(); + assertThat( + kafkaSource + .getConfiguration() + .get(ConfigOptions.key(propertyKey).stringType().noDefaultValue())) + .isEqualTo(propertyValue); + } + + @ParameterizedTest + @MethodSource("provideInvalidCustomDeserializersTestParameters") + public void testSettingInvalidCustomDeserializers( + String propertyKey, String propertyValue, String expectedError) { + assertThatThrownBy(() -> getBasicBuilder().setProperty(propertyKey, propertyValue).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(expectedError); + } + + @Test + public void testDefaultPartitionDiscovery() { + final KafkaSource kafkaSource = getBasicBuilder().build(); + // Commit on checkpoint and auto commit should be disabled because group.id is not specified + assertThat( + kafkaSource + .getConfiguration() + .get(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS)) + .isEqualTo(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.defaultValue()); + } + + @Test + public void testPeriodPartitionDiscovery() { + final KafkaSource kafkaSource = + getBasicBuilder().setBounded(OffsetsInitializer.latest()).build(); + // Commit on checkpoint and auto commit should be disabled because group.id is not specified + assertThat( + kafkaSource + .getConfiguration() + .get(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS)) + .isEqualTo(-1L); + } + private KafkaSourceBuilder getBasicBuilder() { return new KafkaSourceBuilder() .setBootstrapServers("testServer") @@ -206,4 +255,43 @@ public Set getSubscribedTopicPartitions(AdminClient adminClient) return Collections.singleton(new TopicPartition("topic", 0)); } } + + private static Stream provideSettingCustomDeserializerTestParameters() { + return Stream.of( + Arguments.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + TestByteArrayDeserializer.class.getName()), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + TestByteArrayDeserializer.class.getName())); + } + + private static Stream provideInvalidCustomDeserializersTestParameters() { + String deserOne = String.class.getName(); + String deserTwo = "NoneExistentClass"; + String deserThree = StringDeserializer.class.getName(); + return Stream.of( + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserOne, + String.format( + "Deserializer class %s is not a subclass of org.apache.kafka.common.serialization.Deserializer", + deserOne)), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserTwo, + String.format("Deserializer class %s not found", deserTwo)), + Arguments.of( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + deserThree, + String.format( + "Deserializer class %s does not deserialize byte[]", deserThree)), + Arguments.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + deserThree, + String.format( + "Deserializer class %s does not deserialize byte[]", deserThree))); + } + + private class TestByteArrayDeserializer extends ByteArrayDeserializer {} } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index e37aefe0e..6c0bd7e50 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContextFactory; import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; @@ -48,7 +49,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; -import org.apache.flink.util.DockerImageVersions; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; @@ -80,6 +80,7 @@ import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.KafkaTestBase.kafkaServer; import static org.assertj.core.api.Assertions.assertThat; /** Unite test class for {@link KafkaSource}. */ @@ -369,6 +370,40 @@ public void testConsumingTopicWithEmptyPartitions() throws Throwable { WatermarkStrategy.noWatermarks(), "testConsumingTopicWithEmptyPartitions")); } + + @Test + public void testConsumingTransactionalMessage() throws Throwable { + String transactionalTopic = "transactionalTopic-" + UUID.randomUUID(); + KafkaSourceTestEnv.createTestTopic( + transactionalTopic, KafkaSourceTestEnv.NUM_PARTITIONS, 1); + List> records = + KafkaSourceTestEnv.getRecordsForTopic(transactionalTopic); + KafkaSourceTestEnv.produceToKafka( + records, kafkaServer.getTransactionalProducerConfig()); + // After running KafkaSourceTestEnv.setupEarliestOffsets(transactionalTopic): + // - For each partition, records with offsets before partition number P are deleted. + // - Partition 0: offset 0 is earliest + // - Partition 5: offset 5 is earliest, 0-4 are deleted. + // - Partition 9: offset 9 is earliest, 0-8 are deleted. + KafkaSourceTestEnv.setupEarliestOffsets(transactionalTopic); + KafkaSource source = + KafkaSource.builder() + .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) + .setTopics(transactionalTopic) + .setGroupId("topic-with-transactional-message-test") + .setDeserializer(new TestingKafkaRecordDeserializationSchema(false)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setBounded(OffsetsInitializer.latest()) + .build(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + executeAndVerify( + env, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + "testConsumingTransactionalMessage")); + } } /** Integration test based on connector testing framework. */ diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java index 5cc0ddf63..e9c9cab81 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java @@ -23,10 +23,13 @@ import org.apache.flink.streaming.connectors.kafka.KafkaProducerTestBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; +import org.junit.After; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; + /** * An IT case class that runs all the IT cases of the legacy {@link * org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer} with the new {@link KafkaSource}. @@ -44,6 +47,11 @@ public static void prepare() throws Exception { .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); } + @After + public void check() { + checkProducerLeak(); + } + @Test public void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java new file mode 100644 index 000000000..259668c5d --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ + +package org.apache.flink.connector.kafka.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetFacet; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.DefaultTypeDatasetFacet; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.api.lineage.LineageDataset; +import org.apache.flink.streaming.api.lineage.LineageVertex; +import org.apache.flink.util.Collector; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link KafkaSource}. */ +public class KafkaSourceTest { + Properties kafkaProperties; + + @BeforeEach + void setup() { + kafkaProperties = new Properties(); + kafkaProperties.put("bootstrap.servers", "host1;host2"); + } + + @Test + public void testGetLineageVertexWhenSubscriberNotAnKafkaDatasetFacetProvider() { + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber( + new KafkaSubscriber() { + @Override + public Set getSubscribedTopicPartitions( + AdminClient adminClient) { + return null; + } + }) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public TypeInformation getProducedType() { + return null; + } + + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + + assertThat(source.getLineageVertex()) + .extracting(LineageVertex::datasets) + .asList() + .isEmpty(); + } + + @Test + public void testGetLineageVertexWhenNoKafkaTopicsIdentifier() { + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber( + new TestingKafkaSubscriber() { + @Override + public Optional + getDatasetIdentifier() { + return Optional.empty(); + } + }) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + assertThat(source.getLineageVertex()) + .extracting(LineageVertex::datasets) + .asList() + .isEmpty(); + } + + @Test + public void testGetLineageVertex() { + TypeInformation typeInformation = TypeInformation.of(String.class); + KafkaSource source = + new KafkaSourceBuilder() + .setKafkaSubscriber(new TestingKafkaSubscriber()) + .setProperties(kafkaProperties) + .setGroupId("") + .setDeserializer( + new KafkaRecordDeserializationSchema() { + @Override + public void deserialize( + ConsumerRecord record, + Collector out) + throws IOException {} + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + }) + .setUnbounded(OffsetsInitializer.committedOffsets()) + .build(); + + LineageVertex lineageVertex = source.getLineageVertex(); + assertThat(lineageVertex.datasets()).hasSize(1); + LineageDataset dataset = lineageVertex.datasets().get(0); + + assertThat(dataset.namespace()).isEqualTo("kafka://host1"); + assertThat(dataset.name()).isEqualTo("topic1"); + + assertThat(dataset.facets()).containsKey(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME); + DefaultKafkaDatasetFacet kafkaFacet = + (DefaultKafkaDatasetFacet) + dataset.facets().get(DefaultKafkaDatasetFacet.KAFKA_FACET_NAME); + + assertThat(kafkaFacet.getProperties()).containsEntry("bootstrap.servers", "host1;host2"); + + assertThat(dataset.facets()).containsKey(DefaultTypeDatasetFacet.TYPE_FACET_NAME); + assertThat(dataset.facets().get(DefaultTypeDatasetFacet.TYPE_FACET_NAME)) + .hasFieldOrPropertyWithValue("typeInformation", TypeInformation.of(String.class)); + } + + private static class TestingKafkaSubscriber + implements KafkaSubscriber, KafkaDatasetIdentifierProvider { + @Override + public Optional getDatasetIdentifier() { + return Optional.of( + DefaultKafkaDatasetIdentifier.ofTopics(Collections.singletonList("topic1"))); + } + + @Override + public Set getSubscribedTopicPartitions(AdminClient adminClient) { + return null; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index 8d0d3fc11..8b308af16 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SplitsAssignment; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.connector.kafka.source.KafkaSourceOptions; import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; @@ -50,6 +51,7 @@ import java.util.Set; import java.util.StringJoiner; import java.util.regex.Pattern; +import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -64,6 +66,7 @@ public class KafkaEnumeratorTest { private static final int READER0 = 0; private static final int READER1 = 1; + private static final int READER2 = 2; private static final Set PRE_EXISTING_TOPICS = new HashSet<>(Arrays.asList(TOPIC1, TOPIC2)); private static final int PARTITION_DISCOVERY_CALLABLE_INDEX = 0; @@ -226,8 +229,8 @@ public void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws public void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); - // set partitionDiscoveryIntervalMs = 0 - KafkaSourceEnumerator enumerator = createEnumerator(context, 0L)) { + // Disable periodic partition discovery + KafkaSourceEnumerator enumerator = createEnumerator(context, false)) { // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. @@ -254,7 +257,8 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { createEnumerator( context, ENABLE_PERIODIC_PARTITION_DISCOVERY, - INCLUDE_DYNAMIC_TOPIC); + INCLUDE_DYNAMIC_TOPIC, + OffsetsInitializer.latest()); AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { startEnumeratorAndRegisterReaders(context, enumerator); @@ -290,6 +294,19 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { Arrays.asList(READER0, READER1), Collections.singleton(DYNAMIC_TOPIC_NAME), 3); + + // new partitions use EARLIEST_OFFSET, while initial partitions use LATEST_OFFSET + List initialPartitionAssign = + getAllAssignSplits(context, PRE_EXISTING_TOPICS); + assertThat(initialPartitionAssign) + .extracting(KafkaPartitionSplit::getStartingOffset) + .containsOnly((long) KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); + List newPartitionAssign = + getAllAssignSplits(context, Collections.singleton(DYNAMIC_TOPIC_NAME)); + assertThat(newPartitionAssign) + .extracting(KafkaPartitionSplit::getStartingOffset) + .containsOnly(KafkaPartitionSplit.EARLIEST_OFFSET); + } finally { try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { adminClient.deleteTopics(Collections.singleton(DYNAMIC_TOPIC_NAME)).all().get(); @@ -341,9 +358,11 @@ public void testWorkWithPreexistingAssignments() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator( context2, - ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, + OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, preexistingAssignments, + Collections.emptySet(), + true, new Properties())) { enumerator.start(); runPeriodicPartitionDiscovery(context2); @@ -370,9 +389,11 @@ public void testKafkaClientProperties() throws Exception { KafkaSourceEnumerator enumerator = createEnumerator( context, - ENABLE_PERIODIC_PARTITION_DISCOVERY ? 1 : -1, + OffsetsInitializer.earliest(), PRE_EXISTING_TOPICS, Collections.emptySet(), + Collections.emptySet(), + false, properties)) { enumerator.start(); @@ -381,7 +402,7 @@ public void testKafkaClientProperties() throws Exception { assertThat(adminClient).isNotNull(); String clientId = (String) Whitebox.getInternalState(adminClient, "clientId"); assertThat(clientId).isNotNull().startsWith(clientIdPrefix); - assertThat(Whitebox.getInternalState(adminClient, "defaultApiTimeoutMs")) + assertThat((int) Whitebox.getInternalState(adminClient, "defaultApiTimeoutMs")) .isEqualTo(defaultTimeoutMs); assertThat(clientId).isNotNull().startsWith(clientIdPrefix); @@ -395,20 +416,49 @@ public void testSnapshotState() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator(context, false)) { enumerator.start(); - // No reader is registered, so the state should be empty + // Step1: Before first discovery, so the state should be empty final KafkaSourceEnumState state1 = enumerator.snapshotState(1L); assertThat(state1.assignedPartitions()).isEmpty(); + assertThat(state1.unassignedInitialPartitions()).isEmpty(); + assertThat(state1.initialDiscoveryFinished()).isFalse(); registerReader(context, enumerator, READER0); registerReader(context, enumerator, READER1); - runOneTimePartitionDiscovery(context); - // The state should contain splits assigned to READER0 and READER1 - final KafkaSourceEnumState state2 = enumerator.snapshotState(1L); + // Step2: First partition discovery after start, but no assignments to readers + context.runNextOneTimeCallable(); + final KafkaSourceEnumState state2 = enumerator.snapshotState(2L); + assertThat(state2.assignedPartitions()).isEmpty(); + assertThat(state2.unassignedInitialPartitions()).isNotEmpty(); + assertThat(state2.initialDiscoveryFinished()).isTrue(); + + // Step3: Assign partials partitions to reader0 and reader1 + context.runNextOneTimeCallable(); + + // The state should contain splits assigned to READER0 and READER1, but no READER2 + // register. + // Thus, both assignedPartitions and unassignedInitialPartitions are not empty. + final KafkaSourceEnumState state3 = enumerator.snapshotState(3L); verifySplitAssignmentWithPartitions( getExpectedAssignments( new HashSet<>(Arrays.asList(READER0, READER1)), PRE_EXISTING_TOPICS), - state2.assignedPartitions()); + state3.assignedPartitions()); + assertThat(state3.unassignedInitialPartitions()).isNotEmpty(); + assertThat(state3.initialDiscoveryFinished()).isTrue(); + // total partitions of state2 and state3 are equal + // state2 only includes unassignedInitialPartitions + // state3 includes unassignedInitialPartitions + assignedPartitions + Set allPartitionOfState3 = new HashSet<>(); + allPartitionOfState3.addAll(state3.unassignedInitialPartitions()); + allPartitionOfState3.addAll(state3.assignedPartitions()); + assertThat(state2.unassignedInitialPartitions()).isEqualTo(allPartitionOfState3); + + // Step4: register READER2, then all partitions are assigned + registerReader(context, enumerator, READER2); + final KafkaSourceEnumState state4 = enumerator.snapshotState(4L); + assertThat(state4.assignedPartitions()).isEqualTo(allPartitionOfState3); + assertThat(state4.unassignedInitialPartitions()).isEmpty(); + assertThat(state4.initialDiscoveryFinished()).isTrue(); } } @@ -449,6 +499,33 @@ public void testPartitionChangeChecking() throws Throwable { } } + @Test + public void testEnablePartitionDiscoveryByDefault() throws Throwable { + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + KafkaSourceEnumerator enumerator = createEnumerator(context, new Properties())) { + enumerator.start(); + long partitionDiscoveryIntervalMs = + (long) Whitebox.getInternalState(enumerator, "partitionDiscoveryIntervalMs"); + assertThat(partitionDiscoveryIntervalMs) + .isEqualTo(KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.defaultValue()); + assertThat(context.getPeriodicCallables()).isNotEmpty(); + } + } + + @Test + public void testDisablePartitionDiscovery() throws Throwable { + Properties props = new Properties(); + props.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), String.valueOf(0)); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + KafkaSourceEnumerator enumerator = createEnumerator(context, props)) { + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + } + } + // -------------- some common startup sequence --------------- private void startEnumeratorAndRegisterReaders( @@ -480,45 +557,58 @@ private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, boolean enablePeriodicPartitionDiscovery) { return createEnumerator( - enumContext, enablePeriodicPartitionDiscovery, EXCLUDE_DYNAMIC_TOPIC); + enumContext, + enablePeriodicPartitionDiscovery, + EXCLUDE_DYNAMIC_TOPIC, + OffsetsInitializer.earliest()); } private KafkaSourceEnumerator createEnumerator( - MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs) { - return createEnumerator(enumContext, partitionDiscoveryIntervalMs, EXCLUDE_DYNAMIC_TOPIC); + MockSplitEnumeratorContext enumContext, Properties properties) { + return createEnumerator( + enumContext, properties, EXCLUDE_DYNAMIC_TOPIC, OffsetsInitializer.earliest()); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, boolean enablePeriodicPartitionDiscovery, - boolean includeDynamicTopic) { + boolean includeDynamicTopic, + OffsetsInitializer startingOffsetsInitializer) { List topics = new ArrayList<>(PRE_EXISTING_TOPICS); if (includeDynamicTopic) { topics.add(DYNAMIC_TOPIC_NAME); } + Properties props = new Properties(); + props.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + enablePeriodicPartitionDiscovery ? "1" : "-1"); return createEnumerator( enumContext, - enablePeriodicPartitionDiscovery ? 1 : -1, + startingOffsetsInitializer, topics, Collections.emptySet(), - new Properties()); + Collections.emptySet(), + false, + props); } private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs, - boolean includeDynamicTopic) { + Properties props, + boolean includeDynamicTopic, + OffsetsInitializer startingOffsetsInitializer) { List topics = new ArrayList<>(PRE_EXISTING_TOPICS); if (includeDynamicTopic) { topics.add(DYNAMIC_TOPIC_NAME); } return createEnumerator( enumContext, - partitionDiscoveryIntervalMs, + startingOffsetsInitializer, topics, Collections.emptySet(), - new Properties()); + Collections.emptySet(), + false, + props); } /** @@ -527,9 +617,11 @@ private KafkaSourceEnumerator createEnumerator( */ private KafkaSourceEnumerator createEnumerator( MockSplitEnumeratorContext enumContext, - long partitionDiscoveryIntervalMs, + OffsetsInitializer startingOffsetsInitializer, Collection topicsToSubscribe, Set assignedPartitions, + Set unassignedInitialPartitions, + boolean initialDiscoveryFinished, Properties overrideProperties) { // Use a TopicPatternSubscriber so that no exception if a subscribed topic hasn't been // created yet. @@ -538,15 +630,11 @@ private KafkaSourceEnumerator createEnumerator( Pattern topicPattern = Pattern.compile(topicNameJoiner.toString()); KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern); - OffsetsInitializer startingOffsetsInitializer = OffsetsInitializer.earliest(); OffsetsInitializer stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); Properties props = new Properties(KafkaSourceTestEnv.getConsumerProperties(StringDeserializer.class)); KafkaSourceEnumerator.deepCopyProperties(overrideProperties, props); - props.setProperty( - KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), - String.valueOf(partitionDiscoveryIntervalMs)); return new KafkaSourceEnumerator( subscriber, @@ -555,7 +643,8 @@ private KafkaSourceEnumerator createEnumerator( props, enumContext, Boundedness.CONTINUOUS_UNBOUNDED, - assignedPartitions); + new KafkaSourceEnumState( + assignedPartitions, unassignedInitialPartitions, initialDiscoveryFinished)); } // --------------------- @@ -632,6 +721,25 @@ private void verifySplitAssignmentWithPartitions( assertThat(actualTopicPartitions).isEqualTo(allTopicPartitionsFromAssignment); } + /** get all assigned partition splits of topics. */ + private List getAllAssignSplits( + MockSplitEnumeratorContext context, Set topics) { + + List allSplits = new ArrayList<>(); + List> splitsAssignmentSequence = + context.getSplitsAssignmentSequence(); + for (SplitsAssignment splitsAssignment : splitsAssignmentSequence) { + List splitsOfOnceAssignment = + splitsAssignment.assignment().values().stream() + .flatMap(splits -> splits.stream()) + .filter(split -> topics.contains(split.getTopic())) + .collect(Collectors.toList()); + allSplits.addAll(splitsOfOnceAssignment); + } + + return allSplits; + } + private Set asEnumState(Map> assignments) { Set enumState = new HashSet<>(); assignments.forEach( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java index 058d2a095..6c172e4a2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java @@ -44,7 +44,11 @@ public class KafkaSourceEnumStateSerializerTest { @Test public void testEnumStateSerde() throws IOException { - final KafkaSourceEnumState state = new KafkaSourceEnumState(constructTopicPartitions()); + final KafkaSourceEnumState state = + new KafkaSourceEnumState( + constructTopicPartitions(0), + constructTopicPartitions(NUM_PARTITIONS_PER_TOPIC), + true); final KafkaSourceEnumStateSerializer serializer = new KafkaSourceEnumStateSerializer(); final byte[] bytes = serializer.serialize(state); @@ -53,37 +57,53 @@ public void testEnumStateSerde() throws IOException { serializer.deserialize(serializer.getVersion(), bytes); assertThat(restoredState.assignedPartitions()).isEqualTo(state.assignedPartitions()); + assertThat(restoredState.unassignedInitialPartitions()) + .isEqualTo(state.unassignedInitialPartitions()); + assertThat(restoredState.initialDiscoveryFinished()).isTrue(); } @Test public void testBackwardCompatibility() throws IOException { - final Set topicPartitions = constructTopicPartitions(); + final Set topicPartitions = constructTopicPartitions(0); final Map> splitAssignments = toSplitAssignments(topicPartitions); // Create bytes in the way of KafkaEnumStateSerializer version 0 doing serialization - final byte[] bytes = + final byte[] bytesV0 = SerdeUtils.serializeSplitAssignments( splitAssignments, new KafkaPartitionSplitSerializer()); + // Create bytes in the way of KafkaEnumStateSerializer version 1 doing serialization + final byte[] bytesV1 = + KafkaSourceEnumStateSerializer.serializeTopicPartitions(topicPartitions); - // Deserialize above bytes with KafkaEnumStateSerializer version 1 to check backward + // Deserialize above bytes with KafkaEnumStateSerializer version 2 to check backward // compatibility - final KafkaSourceEnumState kafkaSourceEnumState = - new KafkaSourceEnumStateSerializer().deserialize(0, bytes); - - assertThat(kafkaSourceEnumState.assignedPartitions()).isEqualTo(topicPartitions); + final KafkaSourceEnumState kafkaSourceEnumStateV0 = + new KafkaSourceEnumStateSerializer().deserialize(0, bytesV0); + final KafkaSourceEnumState kafkaSourceEnumStateV1 = + new KafkaSourceEnumStateSerializer().deserialize(1, bytesV1); + + assertThat(kafkaSourceEnumStateV0.assignedPartitions()).isEqualTo(topicPartitions); + assertThat(kafkaSourceEnumStateV0.unassignedInitialPartitions()).isEmpty(); + assertThat(kafkaSourceEnumStateV0.initialDiscoveryFinished()).isTrue(); + + assertThat(kafkaSourceEnumStateV1.assignedPartitions()).isEqualTo(topicPartitions); + assertThat(kafkaSourceEnumStateV1.unassignedInitialPartitions()).isEmpty(); + assertThat(kafkaSourceEnumStateV1.initialDiscoveryFinished()).isTrue(); } - private Set constructTopicPartitions() { + private Set constructTopicPartitions(int startPartition) { // Create topic partitions for readers. // Reader i will be assigned with NUM_PARTITIONS_PER_TOPIC splits, with topic name // "topic-{i}" and - // NUM_PARTITIONS_PER_TOPIC partitions. + // NUM_PARTITIONS_PER_TOPIC partitions. The starting partition number is startPartition // Totally NUM_READERS * NUM_PARTITIONS_PER_TOPIC partitions will be created. Set topicPartitions = new HashSet<>(); for (int readerId = 0; readerId < NUM_READERS; readerId++) { - for (int partition = 0; partition < NUM_PARTITIONS_PER_TOPIC; partition++) { + for (int partition = startPartition; + partition < startPartition + NUM_PARTITIONS_PER_TOPIC; + partition++) { topicPartitions.add(new TopicPartition(TOPIC_PREFIX + readerId, partition)); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java index e0cd8506d..46dd61a6f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java @@ -84,7 +84,7 @@ public void testLatestOffsetsInitializer() { assertThat(offsets).hasSameSizeAs(partitions); assertThat(offsets.keySet()).containsAll(partitions); for (long offset : offsets.values()) { - assertThat(offset).isEqualTo(KafkaPartitionSplit.LATEST_OFFSET); + assertThat(offset).isEqualTo(KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); } assertThat(initializer.getAutoOffsetResetStrategy()).isEqualTo(OffsetResetStrategy.LATEST); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java index 258c1c0ab..4c5a50243 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java @@ -18,6 +18,8 @@ package org.apache.flink.connector.kafka.source.enumerator.subscriber; +import org.apache.flink.connector.kafka.lineage.DefaultKafkaDatasetIdentifier; +import org.apache.flink.connector.kafka.lineage.KafkaDatasetIdentifierProvider; import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; import org.apache.kafka.clients.admin.AdminClient; @@ -71,6 +73,8 @@ public void testTopicListSubscriber() { new HashSet<>(KafkaSourceTestEnv.getPartitionsForTopics(topics)); assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofTopics(topics)); } @Test @@ -86,8 +90,8 @@ public void testNonExistingTopic() { @Test public void testTopicPatternSubscriber() { - KafkaSubscriber subscriber = - KafkaSubscriber.getTopicPatternSubscriber(Pattern.compile("pattern.*")); + Pattern pattern = Pattern.compile("pattern.*"); + KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(pattern); final Set subscribedPartitions = subscriber.getSubscribedTopicPartitions(adminClient); @@ -96,6 +100,8 @@ public void testTopicPatternSubscriber() { KafkaSourceTestEnv.getPartitionsForTopics(Collections.singleton(TOPIC2))); assertThat(subscribedPartitions).isEqualTo(expectedSubscribedPartitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofPattern(pattern)); } @Test @@ -111,6 +117,8 @@ public void testPartitionSetSubscriber() { subscriber.getSubscribedTopicPartitions(adminClient); assertThat(subscribedPartitions).isEqualTo(partitions); + assertThat(((KafkaDatasetIdentifierProvider) subscriber).getDatasetIdentifier().get()) + .isEqualTo(DefaultKafkaDatasetIdentifier.ofTopics(topics)); } @Test diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java index 7263bd028..b592a6917 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java @@ -48,6 +48,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.EmptySource; +import org.junit.jupiter.params.provider.NullAndEmptySource; import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; @@ -245,8 +246,8 @@ public void testAssignEmptySplit() throws Exception { final KafkaPartitionSplit emptySplit = new KafkaPartitionSplit( new TopicPartition(TOPIC2, 0), - KafkaPartitionSplit.LATEST_OFFSET, - KafkaPartitionSplit.LATEST_OFFSET); + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION, + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); final KafkaPartitionSplit emptySplitWithZeroStoppingOffset = new KafkaPartitionSplit(new TopicPartition(TOPIC3, 0), 0, 0); @@ -319,6 +320,38 @@ public void testUsingCommittedOffsetsWithEarliestOrLatestOffsetResetStrategy( assertThat(reader.consumer().position(partition)).isEqualTo(expectedOffset); } + @Test + public void testConsumerClientRackSupplier() { + String rackId = "use1-az1"; + Properties properties = new Properties(); + KafkaPartitionSplitReader reader = + createReader( + properties, + UnregisteredMetricsGroup.createSourceReaderMetricGroup(), + rackId); + + // Here we call the helper function directly, because the KafkaPartitionSplitReader + // doesn't allow us to examine the final ConsumerConfig object + reader.setConsumerClientRack(properties, rackId); + assertThat(properties.get(ConsumerConfig.CLIENT_RACK_CONFIG)).isEqualTo(rackId); + } + + @ParameterizedTest + @NullAndEmptySource + public void testSetConsumerClientRackIgnoresNullAndEmpty(String rackId) { + Properties properties = new Properties(); + KafkaPartitionSplitReader reader = + createReader( + properties, + UnregisteredMetricsGroup.createSourceReaderMetricGroup(), + rackId); + + // Here we call the helper function directly, because the KafkaPartitionSplitReader + // doesn't allow us to examine the final ConsumerConfig object + reader.setConsumerClientRack(properties, rackId); + assertThat(properties.containsKey(ConsumerConfig.CLIENT_RACK_CONFIG)).isFalse(); + } + // ------------------ private void assignSplitsAndFetchUntilFinish(KafkaPartitionSplitReader reader, int readerId) @@ -383,6 +416,13 @@ private KafkaPartitionSplitReader createReader() { private KafkaPartitionSplitReader createReader( Properties additionalProperties, SourceReaderMetricGroup sourceReaderMetricGroup) { + return createReader(additionalProperties, sourceReaderMetricGroup, null); + } + + private KafkaPartitionSplitReader createReader( + Properties additionalProperties, + SourceReaderMetricGroup sourceReaderMetricGroup, + String rackId) { Properties props = new Properties(); props.putAll(KafkaSourceTestEnv.getConsumerProperties(ByteArrayDeserializer.class)); props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); @@ -394,7 +434,8 @@ private KafkaPartitionSplitReader createReader( return new KafkaPartitionSplitReader( props, new TestingReaderContext(new Configuration(), sourceReaderMetricGroup), - kafkaSourceReaderMetrics); + kafkaSourceReaderMetrics, + rackId); } private Map assignSplits( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index e4ee39d23..5ad87ffc2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -40,6 +40,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.testutils.MetricListener; import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; +import org.apache.flink.util.function.SerializableSupplier; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; @@ -53,12 +54,14 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -79,6 +82,8 @@ import static org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv.NUM_PARTITIONS; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; /** Unit tests for {@link KafkaSourceReader}. */ public class KafkaSourceReaderTest extends SourceReaderTestBase { @@ -111,7 +116,7 @@ public static void setup() throws Throwable { "Waiting for offsets topic creation failed."); } KafkaSourceTestEnv.produceToKafka( - getRecords(), StringSerializer.class, IntegerSerializer.class); + getRecords(), StringSerializer.class, IntegerSerializer.class, null); } @AfterAll @@ -183,7 +188,26 @@ void testCommitEmptyOffsets() throws Exception { (KafkaSourceReader) createReader(Boundedness.CONTINUOUS_UNBOUNDED, groupId)) { reader.snapshotState(100L); - reader.notifyCheckpointComplete(100L); + reader.snapshotState(101L); + reader.snapshotState(102L); + + // After each snapshot, a new entry should have been added to the offsets-to-commit + // cache for the checkpoint + final Map> expectedOffsetsToCommit = + new HashMap<>(); + expectedOffsetsToCommit.put(100L, new HashMap<>()); + expectedOffsetsToCommit.put(101L, new HashMap<>()); + expectedOffsetsToCommit.put(102L, new HashMap<>()); + assertThat(reader.getOffsetsToCommit()).isEqualTo(expectedOffsetsToCommit); + + // only notify up to checkpoint 101L; all offsets prior to 101L should be evicted from + // cache, leaving only 102L + reader.notifyCheckpointComplete(101L); + + final Map> + expectedOffsetsToCommitAfterNotify = new HashMap<>(); + expectedOffsetsToCommitAfterNotify.put(102L, new HashMap<>()); + assertThat(reader.getOffsetsToCommit()).isEqualTo(expectedOffsetsToCommitAfterNotify); } // Verify the committed offsets. try (AdminClient adminClient = KafkaSourceTestEnv.getAdminClient()) { @@ -271,7 +295,8 @@ void testDisableOffsetCommit() throws Exception { Boundedness.CONTINUOUS_UNBOUNDED, new TestingReaderContext(), (ignore) -> {}, - properties)) { + properties, + null)) { reader.addSplits( getSplits(numSplits, NUM_RECORDS_PER_SPLIT, Boundedness.CONTINUOUS_UNBOUNDED)); ValidatingSourceOutput output = new ValidatingSourceOutput(); @@ -372,7 +397,9 @@ void testAssigningEmptySplits() throws Exception { // Normal split with NUM_RECORDS_PER_SPLIT records final KafkaPartitionSplit normalSplit = new KafkaPartitionSplit( - new TopicPartition(TOPIC, 0), 0, KafkaPartitionSplit.LATEST_OFFSET); + new TopicPartition(TOPIC, 0), + 0, + KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION); // Empty split with no record final KafkaPartitionSplit emptySplit = new KafkaPartitionSplit( @@ -479,6 +506,45 @@ public void testSupportsPausingOrResumingSplits() throws Exception { } } + @Test + public void testThatReaderDoesNotCallRackIdSupplierOnInit() throws Exception { + SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); + + try (KafkaSourceReader reader = + (KafkaSourceReader) + createReader( + Boundedness.CONTINUOUS_UNBOUNDED, + new TestingReaderContext(), + (ignore) -> {}, + new Properties(), + rackIdSupplier)) { + // Do nothing here + } + + verify(rackIdSupplier, never()).get(); + } + + @Test + public void testThatReaderDoesCallRackIdSupplierOnSplitAssignment() throws Exception { + SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); + Mockito.when(rackIdSupplier.get()).thenReturn("use1-az1"); + + try (KafkaSourceReader reader = + (KafkaSourceReader) + createReader( + Boundedness.CONTINUOUS_UNBOUNDED, + new TestingReaderContext(), + (ignore) -> {}, + new Properties(), + rackIdSupplier)) { + reader.addSplits( + Collections.singletonList( + new KafkaPartitionSplit(new TopicPartition(TOPIC, 1), 1L))); + } + + verify(rackIdSupplier).get(); + } + // ------------------------------------------ @Override @@ -535,14 +601,15 @@ private SourceReader createReader( throws Exception { Properties properties = new Properties(); properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return createReader(boundedness, context, splitFinishedHook, properties); + return createReader(boundedness, context, splitFinishedHook, properties, null); } private SourceReader createReader( Boundedness boundedness, SourceReaderContext context, Consumer> splitFinishedHook, - Properties props) + Properties props, + SerializableSupplier rackIdSupplier) throws Exception { KafkaSourceBuilder builder = KafkaSource.builder() @@ -559,6 +626,9 @@ private SourceReader createReader( if (boundedness == Boundedness.BOUNDED) { builder.setBounded(OffsetsInitializer.latest()); } + if (rackIdSupplier != null) { + builder.setRackIdSupplier(rackIdSupplier); + } return KafkaSourceTestUtils.createReaderWithFinishedSplitHook( builder.build(), context, splitFinishedHook); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index 8766719a0..b0ca63161 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -18,18 +18,16 @@ package org.apache.flink.connector.kafka.source.reader.deserializer; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext; import org.apache.flink.formats.json.JsonDeserializationSchema; import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema; import org.apache.flink.util.Collector; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.serialization.StringDeserializer; @@ -38,6 +36,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -82,16 +81,24 @@ public void testKafkaDeserializationSchemaWrapper() throws Exception { @Test public void testKafkaValueDeserializationSchemaWrapper() throws Exception { final ConsumerRecord consumerRecord = getConsumerRecord(); - KafkaRecordDeserializationSchema schema = - KafkaRecordDeserializationSchema.valueOnly( - new JsonDeserializationSchema<>(ObjectNode.class)); + KafkaRecordDeserializationSchema< + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node + .ObjectNode> + schema = + KafkaRecordDeserializationSchema.valueOnly( + new JsonDeserializationSchema<>( + org.apache.flink.shaded.jackson2.com.fasterxml.jackson + .databind.node.ObjectNode.class)); schema.open(new DummyInitializationContext()); - SimpleCollector collector = new SimpleCollector<>(); + SimpleCollector< + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node + .ObjectNode> + collector = new SimpleCollector<>(); schema.deserialize(consumerRecord, collector); assertThat(collector.list).hasSize(1); - ObjectNode deserializedValue = collector.list.get(0); - + org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode + deserializedValue = collector.list.get(0); assertThat(deserializedValue.get("word").asText()).isEqualTo("world"); assertThat(deserializedValue.get("key")).isNull(); assertThat(deserializedValue.get("metadata")).isNull(); @@ -116,7 +123,7 @@ public void testKafkaValueDeserializerWrapper() throws Exception { @Test public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Exception { - final Map config = ImmutableMap.of("simpleKey", "simpleValue"); + final Map config = Collections.singletonMap("simpleKey", "simpleValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly(SimpleStringSerializer.class, config); schema.open(new TestingDeserializationContext()); @@ -127,7 +134,7 @@ public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Except @Test public void testKafkaValueDeserializerWrapperWithConfigurable() throws Exception { - final Map config = ImmutableMap.of("configKey", "configValue"); + final Map config = Collections.singletonMap("configKey", "configValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly( ConfigurableStringSerializer.class, config); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java index 4ca5c9cb6..db7647242 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java @@ -36,11 +36,7 @@ public void testSerializer() throws IOException { Long normalOffset = 1L; TopicPartition topicPartition = new TopicPartition(topic, 1); List stoppingOffsets = - Lists.newArrayList( - KafkaPartitionSplit.COMMITTED_OFFSET, - KafkaPartitionSplit.LATEST_OFFSET, - offsetZero, - normalOffset); + Lists.newArrayList(KafkaPartitionSplit.COMMITTED_OFFSET, offsetZero, normalOffset); KafkaPartitionSplitSerializer splitSerializer = new KafkaPartitionSplitSerializer(); for (Long stoppingOffset : stoppingOffsets) { KafkaPartitionSplit kafkaPartitionSplit = diff --git a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java similarity index 89% rename from flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java rename to flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java index dcc82921f..2d262963b 100644 --- a/flink-connector-kafka-e2e-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/test/DockerImageVersions.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DockerImageVersions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.tests.util.kafka.test; +package org.apache.flink.connector.kafka.testutils; /** * Utility class for defining the image names and versions of Docker containers used during the Java @@ -24,7 +24,9 @@ */ public class DockerImageVersions { - public static final String KAFKA = "confluentinc/cp-kafka:7.2.2"; + public static final String KAFKA = "confluentinc/cp-kafka:7.4.4"; - public static final String SCHEMA_REGISTRY = "confluentinc/cp-schema-registry:7.2.2"; + public static final String SCHEMA_REGISTRY = "confluentinc/cp-schema-registry:7.4.4"; + + public static final String ZOOKEEPER = "zookeeper:3.4.14"; } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java new file mode 100644 index 000000000..a165cc790 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContext.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.streaming.connectors.kafka.DynamicKafkaSourceTestHelper; +import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** A external context for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContext implements DataStreamSourceExternalContext { + private static final Logger logger = + LoggerFactory.getLogger(DynamicKafkaSourceExternalContext.class); + private static final int NUM_TEST_RECORDS_PER_SPLIT = 10; + private static final int NUM_PARTITIONS = 1; + + private static final Pattern STREAM_ID_PATTERN = Pattern.compile("stream-[0-9]+"); + private final List connectorJarPaths; + private final Set kafkaStreams = new HashSet<>(); + private final Map clusterPropertiesMap; + private final List splitDataWriters = new ArrayList<>(); + + // add random suffix to alleviate race conditions with Kafka deleting topics + private final long randomTopicSuffix; + + public DynamicKafkaSourceExternalContext( + List bootstrapServerList, List connectorJarPaths) { + this.connectorJarPaths = connectorJarPaths; + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(0)); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServerList.get(1)); + + this.clusterPropertiesMap = + ImmutableMap.of( + "cluster0", propertiesForCluster0, "cluster1", propertiesForCluster1); + this.randomTopicSuffix = ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) + throws UnsupportedOperationException { + final DynamicKafkaSourceBuilder builder = DynamicKafkaSource.builder(); + + builder.setStreamPattern(STREAM_ID_PATTERN) + .setKafkaMetadataService(new MockKafkaMetadataService(kafkaStreams)) + .setGroupId("DynamicKafkaSourceExternalContext") + .setDeserializer( + KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); + + if (sourceSettings.getBoundedness().equals(Boundedness.BOUNDED)) { + builder.setBounded(OffsetsInitializer.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + int suffix = splitDataWriters.size(); + List> clusterTopics = setupSplits(String.valueOf(suffix)); + SplitDataWriter splitDataWriter = new SplitDataWriter(clusterPropertiesMap, clusterTopics); + this.splitDataWriters.add(splitDataWriter); + return splitDataWriter; + } + + private List> setupSplits(String suffix) { + KafkaStream kafkaStream = getKafkaStream(suffix + randomTopicSuffix); + logger.info("Setting up splits for {}", kafkaStream); + List> clusterTopics = + kafkaStream.getClusterMetadataMap().entrySet().stream() + .flatMap( + entry -> + entry.getValue().getTopics().stream() + .map(topic -> Tuple2.of(entry.getKey(), topic))) + .collect(Collectors.toList()); + + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + KafkaTestEnvironmentImpl.createNewTopic( + topic, NUM_PARTITIONS, 1, clusterPropertiesMap.get(cluster)); + } + + kafkaStreams.add(kafkaStream); + return clusterTopics; + } + + private KafkaStream getKafkaStream(String suffix) { + return new KafkaStream( + "stream-" + suffix, + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0-" + suffix, "topic1-" + suffix), + clusterPropertiesMap.get("cluster0")), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2-" + suffix, "topic3-" + suffix), + clusterPropertiesMap.get("cluster1")))); + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + return IntStream.range(0, NUM_TEST_RECORDS_PER_SPLIT * NUM_PARTITIONS) + .boxed() + .map(num -> Integer.toString(num)) + .collect(Collectors.toList()); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + + @Override + public List getConnectorJarPaths() { + return connectorJarPaths; + } + + @Override + public void close() throws Exception { + // need to clear topics + Map> clusterTopics = new HashMap<>(); + for (SplitDataWriter splitDataWriter : splitDataWriters) { + for (Tuple2 clusterTopic : splitDataWriter.getClusterTopics()) { + clusterTopics + .computeIfAbsent(clusterTopic.f0, unused -> new ArrayList<>()) + .add(clusterTopic.f1); + } + } + for (Map.Entry> entry : clusterTopics.entrySet()) { + String cluster = entry.getKey(); + List topics = entry.getValue(); + try (AdminClient adminClient = AdminClient.create(clusterPropertiesMap.get(cluster))) { + adminClient.deleteTopics(topics).all().get(); + CommonTestUtils.waitUtil( + () -> { + try { + return adminClient.listTopics().listings().get().stream() + .map(TopicListing::name) + .noneMatch(topics::contains); + } catch (Exception e) { + logger.warn("Exception caught when listing Kafka topics", e); + return false; + } + }, + Duration.ofSeconds(30), + String.format("Topics %s were not deleted within timeout", topics)); + } + + logger.info("topics {} are deleted from {}", topics, cluster); + } + } + + private static class SplitDataWriter implements ExternalSystemSplitDataWriter { + private final Map clusterPropertiesMap; + private final List> clusterTopics; + + public SplitDataWriter( + Map clusterPropertiesMap, + List> clusterTopics) { + this.clusterPropertiesMap = clusterPropertiesMap; + this.clusterTopics = clusterTopics; + } + + @Override + public void writeRecords(List records) { + int counter = 0; + try { + for (Tuple2 clusterTopic : clusterTopics) { + String cluster = clusterTopic.f0; + String topic = clusterTopic.f1; + List> producerRecords = new ArrayList<>(); + for (int j = 0; j < NUM_PARTITIONS; j++) { + for (int k = 0; k < NUM_TEST_RECORDS_PER_SPLIT; k++) { + if (records.size() <= counter) { + break; + } + + producerRecords.add( + new ProducerRecord<>(topic, j, null, records.get(counter++))); + } + } + + logger.debug("Writing producer records: {}", producerRecords); + + DynamicKafkaSourceTestHelper.produceToKafka( + clusterPropertiesMap.get(cluster), + producerRecords, + StringSerializer.class, + StringSerializer.class); + } + } catch (Throwable e) { + throw new RuntimeException("Failed to produce test data", e); + } + } + + @Override + public void close() throws Exception {} + + public List> getClusterTopics() { + return clusterTopics; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java new file mode 100644 index 000000000..71798e185 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/DynamicKafkaSourceExternalContextFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.source.DynamicKafkaSource; +import org.apache.flink.connector.testframe.external.ExternalContextFactory; + +import com.google.common.collect.ImmutableList; +import org.testcontainers.containers.KafkaContainer; + +import java.net.URL; +import java.util.List; +import java.util.stream.Collectors; + +/** A external context factory for {@link DynamicKafkaSource} connector testing framework. */ +public class DynamicKafkaSourceExternalContextFactory + implements ExternalContextFactory { + + private final KafkaContainer kafkaContainer0; + private final KafkaContainer kafkaContainer1; + private final List connectorJars; + + public DynamicKafkaSourceExternalContextFactory( + KafkaContainer kafkaContainer0, + KafkaContainer kafkaContainer1, + List connectorJars) { + this.kafkaContainer0 = kafkaContainer0; + this.kafkaContainer1 = kafkaContainer1; + this.connectorJars = connectorJars; + } + + @Override + public DynamicKafkaSourceExternalContext createExternalContext(String testName) { + return new DynamicKafkaSourceExternalContext( + ImmutableList.of( + getBootstrapServers(kafkaContainer0), getBootstrapServers(kafkaContainer1)), + connectorJars); + } + + private static String getBootstrapServers(KafkaContainer kafkaContainer) { + final String internalEndpoints = + kafkaContainer.getNetworkAliases().stream() + .map(host -> String.join(":", host, Integer.toString(9092))) + .collect(Collectors.joining(",")); + return String.join(",", kafkaContainer.getBootstrapServers(), internalEndpoints); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java index 5173f9dc0..d82425f57 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaSourceTestEnv.java @@ -249,7 +249,13 @@ public static void setupCommittedOffsets(String topic) public static void produceToKafka(Collection> records) throws Throwable { - produceToKafka(records, StringSerializer.class, IntegerSerializer.class); + produceToKafka(records, StringSerializer.class, IntegerSerializer.class, null); + } + + public static void produceToKafka( + Collection> records, Properties extraProps) + throws Throwable { + produceToKafka(records, StringSerializer.class, IntegerSerializer.class, extraProps); } public static void setupTopic( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java index 267f7c7c8..78b344b64 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/KafkaUtil.java @@ -18,8 +18,6 @@ package org.apache.flink.connector.kafka.testutils; -import org.apache.flink.util.StringUtils; - import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -35,12 +33,16 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.fail; + /** Collection of methods to interact with a Kafka cluster. */ public class KafkaUtil { @@ -49,24 +51,34 @@ public class KafkaUtil { private KafkaUtil() {} - /** - * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log - * levels with the ones used by the capturing logger. - * - * @param dockerImageVersion describing the Kafka image - * @param logger to derive the log level from - * @return configured Kafka container - */ - public static KafkaContainer createKafkaContainer(String dockerImageVersion, Logger logger) { - return createKafkaContainer(dockerImageVersion, logger, null); + /** This method helps to set commonly used Kafka configurations and sets up the logger. */ + public static KafkaContainer createKafkaContainer(Class testCase) { + return createKafkaContainer(getContainerName("kafka", testCase)); } - /** - * This method helps to set commonly used Kafka configurations and aligns the internal Kafka log - * levels with the ones used by the capturing logger, and set the prefix of logger. - */ - public static KafkaContainer createKafkaContainer( - String dockerImageVersion, Logger logger, String loggerPrefix) { + /** This method helps to set commonly used Kafka configurations and sets up the logger. */ + public static KafkaContainer createKafkaContainer(String containerName) { + Logger logger = getLogger(containerName); + + String logLevel = inferLogLevel(logger); + + Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(logger, true); + return new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) + .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") + .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") + .withEnv("KAFKA_LOG4J_ROOT_LOGLEVEL", logLevel) + .withEnv("KAFKA_LOG4J_LOGGERS", "state.change.logger=" + logLevel) + .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") + .withEnv( + "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", + String.valueOf(Duration.ofHours(2).toMillis())) + .withEnv("KAFKA_LOG4J_TOOLS_ROOT_LOGLEVEL", logLevel) + .withLogConsumer(logConsumer); + } + + private static String inferLogLevel(Logger logger) { String logLevel; if (logger.isTraceEnabled()) { logLevel = "TRACE"; @@ -81,24 +93,19 @@ public static KafkaContainer createKafkaContainer( } else { logLevel = "OFF"; } + return logLevel; + } - Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(logger); - if (!StringUtils.isNullOrWhitespaceOnly(loggerPrefix)) { - logConsumer.withPrefix(loggerPrefix); - } - return new KafkaContainer(DockerImageName.parse(dockerImageVersion)) - .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") - .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") - .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") - .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") - .withEnv("KAFKA_LOG4J_ROOT_LOGLEVEL", logLevel) - .withEnv("KAFKA_LOG4J_LOGGERS", "state.change.logger=" + logLevel) - .withEnv("KAFKA_CONFLUENT_SUPPORT_METRICS_ENABLE", "false") - .withEnv( - "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", - String.valueOf(Duration.ofHours(2).toMillis())) - .withEnv("KAFKA_LOG4J_TOOLS_ROOT_LOGLEVEL", logLevel) - .withLogConsumer(logConsumer); + public static Logger getLogger(String containerName) { + return LoggerFactory.getLogger("container." + containerName); + } + + public static Logger getLogger(String type, Class testClass) { + return getLogger(getContainerName(type, testClass)); + } + + private static String getContainerName(String type, Class testClass) { + return type + "." + testClass.getSimpleName(); } /** @@ -186,4 +193,49 @@ private static Set getAllPartitions( .map(info -> new TopicPartition(info.topic(), info.partition())) .collect(Collectors.toSet()); } + + private static final Set KNOWN_LEAKS = new ConcurrentSkipListSet<>(); + + public static void checkProducerLeak() { + List> leaks = null; + for (int tries = 0; tries < 10; tries++) { + leaks = + Thread.getAllStackTraces().entrySet().stream() + .filter(KafkaUtil::findAliveKafkaThread) + .filter( + threadEntry -> + !KNOWN_LEAKS.contains(threadEntry.getKey().getId())) + .collect(Collectors.toList()); + if (leaks.isEmpty()) { + return; + } + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + } + + for (Map.Entry leak : leaks) { + KNOWN_LEAKS.add(leak.getKey().getId()); + } + fail( + "Detected new producer leaks:\n" + + leaks.stream() + .map(KafkaUtil::format) + .collect(Collectors.joining("\n\n"))); + } + + private static String format(Map.Entry leak) { + String stackTrace = + Arrays.stream(leak.getValue()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n")); + return leak.getKey().getName() + ":\n" + stackTrace; + } + + private static boolean findAliveKafkaThread( + Map.Entry threadStackTrace) { + return threadStackTrace.getKey().getState() != Thread.State.TERMINATED + && threadStackTrace.getKey().getName().contains("kafka-producer-network-thread"); + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java new file mode 100644 index 000000000..18854cf90 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/MockKafkaMetadataService.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** A mock in-memory implementation of {@link KafkaMetadataService}. */ +public class MockKafkaMetadataService implements KafkaMetadataService { + + private Set kafkaStreams; + private Set kafkaClusterIds; + private boolean throwException = false; + + public MockKafkaMetadataService(boolean throwException) { + this.throwException = throwException; + } + + public MockKafkaMetadataService(Set kafkaStreams) { + setKafkaStreams(kafkaStreams); + } + + public void setKafkaStreams(Set kafkaStreams) { + this.kafkaStreams = kafkaStreams; + this.kafkaClusterIds = + kafkaStreams.stream() + .flatMap( + kafkaStream -> + kafkaStream.getClusterMetadataMap().keySet().stream()) + .collect(Collectors.toSet()); + } + + public void setThrowException(boolean throwException) { + this.throwException = throwException; + } + + private void checkAndThrowException() { + if (throwException) { + throw new RuntimeException("Mock exception"); + } + } + + @Override + public Set getAllStreams() { + checkAndThrowException(); + return kafkaStreams; + } + + @Override + public Map describeStreams(Collection streamIds) { + checkAndThrowException(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (KafkaStream stream : getAllStreams()) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + @Override + public boolean isClusterActive(String kafkaClusterId) { + checkAndThrowException(); + return kafkaClusterIds.contains(kafkaClusterId); + } + + @Override + public void close() throws Exception {} +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java new file mode 100644 index 000000000..6c5036a9b --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/TwoKafkaContainers.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +/** Wraps 2 Kafka containers into one for test utilities that only accept one container. */ +public class TwoKafkaContainers extends GenericContainer { + private final KafkaContainer kafka0; + private final KafkaContainer kafka1; + + public TwoKafkaContainers() { + DockerImageName dockerImageName = DockerImageName.parse(DockerImageVersions.KAFKA); + this.kafka0 = new KafkaContainer(dockerImageName); + this.kafka1 = new KafkaContainer(dockerImageName); + } + + @Override + public boolean isRunning() { + return kafka0.isRunning() && kafka1.isRunning(); + } + + @Override + public void start() { + kafka0.start(); + kafka1.start(); + } + + @Override + public void stop() { + kafka0.stop(); + kafka1.stop(); + } + + public KafkaContainer getKafka0() { + return kafka0; + } + + public KafkaContainer getKafka1() { + return kafka1; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java new file mode 100644 index 000000000..4a1dab17a --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataService.java @@ -0,0 +1,367 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaMetadataService; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.CommonClientConfigs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.DumperOptions; +import org.yaml.snakeyaml.LoaderOptions; +import org.yaml.snakeyaml.TypeDescription; +import org.yaml.snakeyaml.Yaml; +import org.yaml.snakeyaml.constructor.Constructor; +import org.yaml.snakeyaml.nodes.Node; +import org.yaml.snakeyaml.nodes.SequenceNode; +import org.yaml.snakeyaml.nodes.Tag; +import org.yaml.snakeyaml.representer.Representer; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +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.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Reads metadata from yaml file and lazily refreshes periodically. This implementation assumes that + * specified topics exist in the clusters that are contained in the yaml metadata. Therefore, topic + * is used as the stream name. This is designed to integrate with K8s configmap and cluster + * migration. + * + *

      Files must be of the form: + * + *

      {@code
      + * - streamId: stream0
      + *   clusterMetadataList:
      + *     - clusterId: cluster0
      + *       bootstrapServers: bootstrap-server-0:443
      + *       topics:
      + *         - topic0
      + *         - topic1
      + *     - clusterId: cluster1
      + *       bootstrapServers: bootstrap-server-1:443
      + *       topics:
      + *         - topic2
      + *         - topic3
      + * - streamId: stream1
      + *   clusterMetadataList:
      + *     - clusterId: cluster2
      + *       bootstrapServers: bootstrap-server-2:443
      + *       topics:
      + *         - topic4
      + *         - topic5
      + * }
      + * + *

      Typically, usage will look like: first consuming from one cluster, second adding new cluster + * and consuming from both clusters, and third consuming from only from the new cluster after all + * data from the old cluster has been read. + */ +public class YamlFileMetadataService implements KafkaMetadataService { + private static final Logger logger = LoggerFactory.getLogger(YamlFileMetadataService.class); + private final String metadataFilePath; + private final Duration refreshInterval; + private Instant lastRefresh; + // current metadata should be accessed from #getAllStreams() + private transient Set streamMetadata; + private transient Yaml yaml; + + /** + * Constructs a metadata service based on cluster information stored in a file. + * + * @param metadataFilePath location of the metadata file + * @param metadataTtl ttl of metadata that controls how often to refresh + */ + public YamlFileMetadataService(String metadataFilePath, Duration metadataTtl) { + this.metadataFilePath = metadataFilePath; + this.refreshInterval = metadataTtl; + this.lastRefresh = Instant.MIN; + } + + /** + * {@inheritDoc} + * + *

      This obtains the all stream metadata and enforces the ttl configuration on the metadata. + */ + @Override + public Set getAllStreams() { + refreshIfNeeded(); + return streamMetadata; + } + + /** {@inheritDoc} */ + @Override + public Map describeStreams(Collection streamIds) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + Set streams = getAllStreams(); + for (KafkaStream stream : streams) { + if (streamIds.contains(stream.getStreamId())) { + builder.put(stream.getStreamId(), stream); + } + } + + return builder.build(); + } + + /** {@inheritDoc} */ + @Override + public boolean isClusterActive(String kafkaClusterId) { + return getAllStreams().stream() + .flatMap(kafkaStream -> kafkaStream.getClusterMetadataMap().keySet().stream()) + .anyMatch(cluster -> cluster.equals(kafkaClusterId)); + } + + /** {@inheritDoc} */ + @Override + public void close() throws Exception {} + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param streamMetadata list of {@link StreamMetadata} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYaml(List streamMetadata, File metadataFile) + throws IOException { + logger.debug("Writing stream infos to file: {}", streamMetadata); + Yaml yaml = initYamlParser(); + FileWriter fileWriter = new FileWriter(metadataFile, false); + yaml.dump(streamMetadata, fileWriter); + fileWriter.close(); + } + + /** + * A utility method for writing metadata in the expected yaml format. + * + * @param kafkaStreams list of {@link KafkaStream} + * @param metadataFile the metadata {@link File} + */ + public static void saveToYamlFromKafkaStreams(List kafkaStreams, File metadataFile) + throws IOException { + saveToYaml( + kafkaStreams.stream() + .map(YamlFileMetadataService::convertToStreamMetadata) + .collect(Collectors.toList()), + metadataFile); + } + + private static StreamMetadata convertToStreamMetadata(KafkaStream kafkaStream) { + return new StreamMetadata( + kafkaStream.getStreamId(), + kafkaStream.getClusterMetadataMap().entrySet().stream() + .map( + entry -> + new StreamMetadata.ClusterMetadata( + entry.getKey(), + entry.getValue() + .getProperties() + .getProperty( + CommonClientConfigs + .BOOTSTRAP_SERVERS_CONFIG), + new ArrayList<>(entry.getValue().getTopics()))) + .collect(Collectors.toList())); + } + + private void refreshIfNeeded() { + Instant now = Instant.now(); + try { + if (now.isAfter(lastRefresh.plus(refreshInterval.toMillis(), ChronoUnit.MILLIS))) { + streamMetadata = parseFile(); + lastRefresh = now; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @VisibleForTesting + Set parseFile() throws IOException { + if (yaml == null) { + yaml = initYamlParser(); + } + + List streamMetadataList = + yaml.load(Files.newInputStream(Paths.get(metadataFilePath))); + if (logger.isDebugEnabled()) { + logger.debug( + "Input stream of metadata file has size: {}", + Files.newInputStream(Paths.get(metadataFilePath)).available()); + } + Set kafkaStreams = new HashSet<>(); + + for (StreamMetadata streamMetadata : streamMetadataList) { + Map clusterMetadataMap = new HashMap<>(); + + for (StreamMetadata.ClusterMetadata clusterMetadata : + streamMetadata.getClusterMetadataList()) { + final String kafkaClusterId; + if (clusterMetadata.getClusterId() != null) { + kafkaClusterId = clusterMetadata.getClusterId(); + } else { + kafkaClusterId = clusterMetadata.getBootstrapServers(); + } + + Properties properties = new Properties(); + properties.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + clusterMetadata.getBootstrapServers()); + clusterMetadataMap.put( + kafkaClusterId, + new ClusterMetadata( + new HashSet<>(clusterMetadata.getTopics()), properties)); + } + + kafkaStreams.add(new KafkaStream(streamMetadata.getStreamId(), clusterMetadataMap)); + } + + logger.debug("From {} loaded metadata: {}", metadataFilePath, kafkaStreams); + return kafkaStreams; + } + + private static Yaml initYamlParser() { + DumperOptions dumperOptions = new DumperOptions(); + Representer representer = new Representer(dumperOptions); + representer.addClassTag(StreamMetadata.class, Tag.MAP); + TypeDescription typeDescription = new TypeDescription(StreamMetadata.class); + representer.addTypeDescription(typeDescription); + representer.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK); + LoaderOptions loaderOptions = new LoaderOptions(); + // Allow global tag for StreamMetadata + loaderOptions.setTagInspector( + tag -> tag.getClassName().equals(StreamMetadata.class.getName())); + return new Yaml(new ListConstructor<>(StreamMetadata.class, loaderOptions), representer); + } + + /** A custom constructor is required to read yaml lists at the root. */ + private static class ListConstructor extends Constructor { + private final Class clazz; + + public ListConstructor(final Class clazz, final LoaderOptions loaderOptions) { + super(loaderOptions); + this.clazz = clazz; + } + + @Override + protected Object constructObject(final Node node) { + if (node instanceof SequenceNode && isRootNode(node)) { + ((SequenceNode) node).setListType(clazz); + } + return super.constructObject(node); + } + + private boolean isRootNode(final Node node) { + return node.getStartMark().getIndex() == 0; + } + } + + /** Internal class for snake yaml parsing. A mutable, no arg, public class is required. */ + public static class StreamMetadata { + + private String streamId; + private List clusterMetadataList; + + public StreamMetadata() {} + + public StreamMetadata(String streamId, List clusterMetadataList) { + this.streamId = streamId; + this.clusterMetadataList = clusterMetadataList; + } + + public String getStreamId() { + return streamId; + } + + public void setStreamId(String streamId) { + this.streamId = streamId; + } + + public List getClusterMetadataList() { + return clusterMetadataList; + } + + public void setClusterMetadataList(List clusterMetadata) { + this.clusterMetadataList = clusterMetadata; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("streamId", streamId) + .add("clusterMetadataList", clusterMetadataList) + .toString(); + } + + /** Information to connect to a particular cluster. */ + public static class ClusterMetadata { + private String clusterId; + private String bootstrapServers; + private List topics; + + public ClusterMetadata() {} + + public ClusterMetadata(String clusterId, String bootstrapServers, List topics) { + this.clusterId = clusterId; + this.bootstrapServers = bootstrapServers; + this.topics = topics; + } + + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public List getTopics() { + return topics; + } + + public void setTopics(List topics) { + this.topics = topics; + } + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java new file mode 100644 index 000000000..f0012d181 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/testutils/YamlFileMetadataServiceTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.connector.kafka.testutils; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.Resources; +import org.apache.kafka.clients.CommonClientConfigs; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.Properties; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** A test class for {@link YamlFileMetadataService}. */ +public class YamlFileMetadataServiceTest { + + @Test + public void testParseFile() throws IOException { + YamlFileMetadataService yamlFileMetadataService = + new YamlFileMetadataService( + Resources.getResource("stream-metadata.yaml").getPath(), Duration.ZERO); + Set kafkaStreams = yamlFileMetadataService.parseFile(); + + Properties propertiesForCluster0 = new Properties(); + propertiesForCluster0.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-0:443"); + Properties propertiesForCluster1 = new Properties(); + propertiesForCluster1.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-1:443"); + Properties propertiesForCluster2 = new Properties(); + propertiesForCluster2.setProperty( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "bootstrap-server-2:443"); + + assertThat(kafkaStreams) + .containsExactlyInAnyOrderElementsOf( + ImmutableSet.of( + new KafkaStream( + "stream0", + ImmutableMap.of( + "cluster0", + new ClusterMetadata( + ImmutableSet.of("topic0", "topic1"), + propertiesForCluster0), + "cluster1", + new ClusterMetadata( + ImmutableSet.of("topic2", "topic3"), + propertiesForCluster1))), + new KafkaStream( + "stream1", + ImmutableMap.of( + "cluster2", + new ClusterMetadata( + ImmutableSet.of("topic4", "topic5"), + propertiesForCluster2))))); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java new file mode 100644 index 000000000..8eb0d28c1 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/DynamicKafkaSourceTestHelper.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.connector.kafka.dynamic.metadata.ClusterMetadata; +import org.apache.flink.connector.kafka.dynamic.metadata.KafkaStream; +import org.apache.flink.connector.kafka.dynamic.source.MetadataUpdateEvent; + +import org.apache.kafka.clients.producer.Callback; +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.common.header.internals.RecordHeader; +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** Brings up multiple kafka clusters and provides utilities to setup test data. */ +public class DynamicKafkaSourceTestHelper extends KafkaTestBase { + public static final int NUM_KAFKA_CLUSTERS = 2; + + public static void setup() throws Throwable { + setNumKafkaClusters(NUM_KAFKA_CLUSTERS); + prepare(); + } + + public static void tearDown() throws Exception { + shutDownServices(); + } + + public static KafkaClusterTestEnvMetadata getKafkaClusterTestEnvMetadata(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx); + } + + public static MetadataUpdateEvent getMetadataUpdateEvent(String topic) { + return new MetadataUpdateEvent(Collections.singleton(getKafkaStream(topic))); + } + + public static String getKafkaClusterId(int kafkaClusterIdx) { + return kafkaClusters.get(kafkaClusterIdx).getKafkaClusterId(); + } + + public static Map getClusterMetadataMap( + int kafkaClusterIdx, String... topics) { + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + getKafkaClusterTestEnvMetadata(kafkaClusterIdx); + + Set topicsSet = new HashSet<>(Arrays.asList(topics)); + + ClusterMetadata clusterMetadata = + new ClusterMetadata(topicsSet, kafkaClusterTestEnvMetadata.getStandardProperties()); + + return Collections.singletonMap( + kafkaClusterTestEnvMetadata.getKafkaClusterId(), clusterMetadata); + } + + /** Stream is a topic across multiple clusters. */ + public static KafkaStream getKafkaStream(String topic) { + Map clusterMetadataMap = new HashMap<>(); + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + clusterMetadataMap.putAll(getClusterMetadataMap(i, topic)); + } + + return new KafkaStream(topic, clusterMetadataMap); + } + + public static void createTopic(String topic, int numPartitions, int replicationFactor) { + for (int i = 0; i < NUM_KAFKA_CLUSTERS; i++) { + createTopic(i, topic, numPartitions, replicationFactor); + } + } + + public static void createTopic(String topic, int numPartitions) { + createTopic(topic, numPartitions, 1); + } + + public static void createTopic(int kafkaClusterIdx, String topic, int numPartitions) { + createTopic(kafkaClusterIdx, topic, numPartitions, 1); + } + + private static void createTopic( + int kafkaClusterIdx, String topic, int numPartitions, int replicationFactor) { + kafkaClusters + .get(kafkaClusterIdx) + .getKafkaTestEnvironment() + .createTestTopic(topic, numPartitions, replicationFactor); + } + + /** Produces [0, numPartitions*numRecordsPerSplit) range of records to the specified topic. */ + public static List> produceToKafka( + String topic, int numPartitions, int numRecordsPerSplit) throws Throwable { + List> records = new ArrayList<>(); + + int counter = 0; + for (int kafkaClusterIdx = 0; kafkaClusterIdx < NUM_KAFKA_CLUSTERS; kafkaClusterIdx++) { + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes( + StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + records.addAll(recordsForCluster); + } + + return records; + } + + /** + * Produces [recordValueStartingOffset, recordValueStartingOffset + + * numPartitions*numRecordsPerSplit) range of records to the specified topic and cluster. + */ + public static int produceToKafka( + int kafkaClusterIdx, + String topic, + int numPartitions, + int numRecordsPerSplit, + int recordValueStartingOffset) + throws Throwable { + int counter = recordValueStartingOffset; + String kafkaClusterId = getKafkaClusterId(kafkaClusterIdx); + List> recordsForCluster = new ArrayList<>(); + for (int part = 0; part < numPartitions; part++) { + for (int i = 0; i < numRecordsPerSplit; i++) { + recordsForCluster.add( + new ProducerRecord<>( + topic, + part, + topic + "-" + part, + counter++, + Collections.singleton( + new RecordHeader( + "flink.kafka-cluster-name", + kafkaClusterId.getBytes(StandardCharsets.UTF_8))))); + } + } + + produceToKafka(kafkaClusterIdx, recordsForCluster); + + return counter; + } + + public static void produceToKafka( + int kafkaClusterIdx, Collection> records) + throws Throwable { + produceToKafka(kafkaClusterIdx, records, StringSerializer.class, IntegerSerializer.class); + } + + public static void produceToKafka( + int id, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + produceToKafka( + kafkaClusters.get(id).getStandardProperties(), + records, + keySerializerClass, + valueSerializerClass); + } + + public static void produceToKafka( + Properties clusterProperties, + Collection> records, + Class> keySerializerClass, + Class> + valueSerializerClass) + throws Throwable { + Properties props = new Properties(); + props.putAll(clusterProperties); + props.setProperty(ProducerConfig.ACKS_CONFIG, "all"); + props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName()); + props.setProperty( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); + + AtomicReference sendingError = new AtomicReference<>(); + Callback callback = + (metadata, exception) -> { + if (exception != null) { + if (!sendingError.compareAndSet(null, exception)) { + sendingError.get().addSuppressed(exception); + } + } + }; + try (KafkaProducer producer = new KafkaProducer<>(props)) { + for (ProducerRecord record : records) { + producer.send(record, callback); + } + producer.flush(); + } + if (sendingError.get() != null) { + throw sendingError.get(); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 296545cad..47bce8bd9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -92,7 +92,7 @@ public class FlinkKafkaConsumerBaseMigrationTest { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { - return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); + return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.current()); } public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java new file mode 100644 index 000000000..90c773730 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.test.util.MiniClusterWithClientResource; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.junit.ClassRule; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Path; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; + +/** ITCase tests class for {@link FlinkKafkaConsumer}. */ +@TestInstance(Lifecycle.PER_CLASS) +public class FlinkKafkaConsumerITCase { + private static final String TOPIC1 = "FlinkKafkaConsumerITCase_topic1"; + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(new Configuration()) + .build()); + + @BeforeAll + public void setup() throws Throwable { + KafkaSourceTestEnv.setup(); + KafkaSourceTestEnv.setupTopic( + TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopicWithoutTimestamp); + } + + @AfterAll + public void tearDown() throws Exception { + KafkaSourceTestEnv.tearDown(); + } + + @Test + public void testStopWithSavepoint(@TempDir Path savepointsDir) throws Exception { + Configuration config = + new Configuration() + .set( + CheckpointingOptions.SAVEPOINT_DIRECTORY, + savepointsDir.toUri().toString()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(1); + + Properties properties = new Properties(); + properties.setProperty( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + KafkaSourceTestEnv.brokerConnectionStrings); + properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testStopWithSavepoint"); + properties.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + FlinkKafkaConsumer kafkaConsumer = + new FlinkKafkaConsumer<>( + TOPIC1, + new TypeInformationSerializationSchema<>( + BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig()), + properties); + DataStreamSource stream = env.addSource(kafkaConsumer); + + ProgressLatchingIdentityFunction.resetBeforeUse(); + stream.map(new ProgressLatchingIdentityFunction()).addSink(new DiscardingSink<>()); + + JobClient jobClient = env.executeAsync(); + + ProgressLatchingIdentityFunction.getProgressLatch().await(); + + // Check that stopWithSavepoint completes successfully + jobClient.stopWithSavepoint(false, null, SavepointFormatType.CANONICAL).get(); + // TODO: ideally we should test recovery, that there were no data losses etc, but this + // is already a deprecated class, so I'm not adding new tests for that now. + } + + private static class ProgressLatchingIdentityFunction implements MapFunction { + + static CountDownLatch progressLatch; + + static void resetBeforeUse() { + progressLatch = new CountDownLatch(1); + } + + public static CountDownLatch getProgressLatch() { + return progressLatch; + } + + @Override + public Integer map(Integer integer) throws Exception { + progressLatch.countDown(); + return integer; + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java index 2f393374a..15729a8c8 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java @@ -20,8 +20,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -35,6 +33,7 @@ import java.time.Duration; import java.util.Collections; +import java.util.Iterator; import java.util.Properties; import java.util.UUID; @@ -241,9 +240,11 @@ private void assertRecord(String topicName, String expectedKey, String expectedV records = kafkaConsumer.poll(Duration.ofMillis(10000)); } - ConsumerRecord record = Iterables.getOnlyElement(records); + final Iterator> it = records.iterator(); + ConsumerRecord record = it.next(); assertThat(record.key()).isEqualTo(expectedKey); assertThat(record.value()).isEqualTo(expectedValue); + assertThat(it.hasNext()).isFalse(); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java deleted file mode 100644 index 4274fcff2..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java +++ /dev/null @@ -1,482 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.testutils.CheckedThread; -import org.apache.flink.core.testutils.MultiShotLatch; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.functions.sink.SinkContextUtil; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.internals.KeyedSerializationSchemaWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; - -import org.apache.kafka.clients.producer.Callback; -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.common.PartitionInfo; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -/** Tests for the {@link FlinkKafkaProducerBase}. */ -public class FlinkKafkaProducerBaseTest { - - /** Tests that the constructor eagerly checks bootstrap servers are set in config. */ - @Test(expected = IllegalArgumentException.class) - public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception { - // no bootstrap servers set in props - Properties props = new Properties(); - // should throw IllegalArgumentException - new DummyFlinkKafkaProducer<>( - props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); - } - - /** - * Tests that constructor defaults to key value serializers in config to byte array - * deserializers if not set. - */ - @Test - public void testKeyValueDeserializersSetIfMissing() throws Exception { - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345"); - // should set missing key value deserializers - new DummyFlinkKafkaProducer<>( - props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); - - assertThat(props) - .containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG) - .containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); - assertThat(props.getProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) - .isEqualTo(ByteArraySerializer.class.getName()); - assertThat(props.getProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) - .isEqualTo(ByteArraySerializer.class.getName()); - } - - /** Tests that partitions list is determinate and correctly provided to custom partitioner. */ - @SuppressWarnings("unchecked") - @Test - public void testPartitionerInvokedWithDeterminatePartitionList() throws Exception { - FlinkKafkaPartitioner mockPartitioner = mock(FlinkKafkaPartitioner.class); - - RuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); - when(mockRuntimeContext.getIndexOfThisSubtask()).thenReturn(0); - when(mockRuntimeContext.getNumberOfParallelSubtasks()).thenReturn(1); - - // out-of-order list of 4 partitions - List mockPartitionsList = new ArrayList<>(4); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 3, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 1, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 0, null, null, null)); - mockPartitionsList.add( - new PartitionInfo(DummyFlinkKafkaProducer.DUMMY_TOPIC, 2, null, null, null)); - - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - mockPartitioner); - producer.setRuntimeContext(mockRuntimeContext); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - when(mockProducer.partitionsFor(anyString())).thenReturn(mockPartitionsList); - when(mockProducer.metrics()).thenReturn(null); - - producer.open(new Configuration()); - verify(mockPartitioner, times(1)).open(0, 1); - - producer.invoke("foobar", SinkContextUtil.forTimestamp(0)); - verify(mockPartitioner, times(1)) - .partition( - "foobar", - null, - "foobar".getBytes(), - DummyFlinkKafkaProducer.DUMMY_TOPIC, - new int[] {0, 1, 2, 3}); - } - - /** - * Test ensuring that if an invoke call happens right after an async exception is caught, it - * should be rethrown. - */ - @Test - public void testAsyncErrorRethrownOnInvoke() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - - OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - - // let the message request return an async exception - producer.getPendingCallbacks() - .get(0) - .onCompletion(null, new Exception("artificial async exception")); - - try { - testHarness.processElement(new StreamRecord<>("msg-2")); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that if a snapshot call happens right after an async exception is caught, it - * should be rethrown. - */ - @Test - public void testAsyncErrorRethrownOnCheckpoint() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - - OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - - // let the message request return an async exception - producer.getPendingCallbacks() - .get(0) - .onCompletion(null, new Exception("artificial async exception")); - - try { - testHarness.snapshot(123L, 123L); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that if an async exception is caught for one of the flushed requests on - * checkpoint, it should be rethrown; we set a timeout because the test will not finish if the - * logic is broken. - * - *

      Note that this test does not test the snapshot method is blocked correctly when there are - * pending records. The test for that is covered in testAtLeastOnceProducer. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(true); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - - verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class)); - - // only let the first callback succeed for now - producer.getPendingCallbacks().get(0).onCompletion(null, null); - - CheckedThread snapshotThread = - new CheckedThread() { - @Override - public void go() throws Exception { - // this should block at first, since there are still two pending records - // that needs to be flushed - testHarness.snapshot(123L, 123L); - } - }; - snapshotThread.start(); - - // let the 2nd message fail with an async exception - producer.getPendingCallbacks() - .get(1) - .onCompletion(null, new Exception("artificial async failure for 2nd message")); - producer.getPendingCallbacks().get(2).onCompletion(null, null); - - try { - snapshotThread.sync(); - } catch (Exception e) { - // the snapshot should have failed with the async exception - assertThat(e.getCause().getMessage()) - .contains("artificial async failure for 2nd message"); - - // test succeeded - return; - } - - fail("unknown failure"); - } - - /** - * Test ensuring that the producer is not dropping buffered records; we set a timeout because - * the test will not finish if the logic is broken. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 10000) - public void testAtLeastOnceProducer() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(true); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg-1")); - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - - verify(mockProducer, times(3)).send(any(ProducerRecord.class), any(Callback.class)); - assertThat(producer.getPendingSize()).isEqualTo(3); - - // start a thread to perform checkpointing - CheckedThread snapshotThread = - new CheckedThread() { - @Override - public void go() throws Exception { - // this should block until all records are flushed; - // if the snapshot implementation returns before pending records are - // flushed, - testHarness.snapshot(123L, 123L); - } - }; - snapshotThread.start(); - - // before proceeding, make sure that flushing has started and that the snapshot is still - // blocked; - // this would block forever if the snapshot didn't perform a flush - producer.waitUntilFlushStarted(); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - - // now, complete the callbacks - producer.getPendingCallbacks().get(0).onCompletion(null, null); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - assertThat(producer.getPendingSize()).isEqualTo(2); - - producer.getPendingCallbacks().get(1).onCompletion(null, null); - assertThat(snapshotThread.isAlive()) - .as("Snapshot returned before all records were flushed") - .isTrue(); - assertThat(producer.getPendingSize()).isEqualTo(1); - - producer.getPendingCallbacks().get(2).onCompletion(null, null); - assertThat(producer.getPendingSize()).isEqualTo(0); - - // this would fail with an exception if flushing wasn't completed before the snapshot method - // returned - snapshotThread.sync(); - - testHarness.close(); - } - - /** - * This test is meant to assure that testAtLeastOnceProducer is valid by testing that if - * flushing is disabled, the snapshot method does indeed finishes without waiting for pending - * records; we set a timeout because the test will not finish if the logic is broken. - */ - @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { - final DummyFlinkKafkaProducer producer = - new DummyFlinkKafkaProducer<>( - FakeStandardProducerConfig.get(), - new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), - null); - producer.setFlushOnCheckpoint(false); - - final KafkaProducer mockProducer = producer.getMockKafkaProducer(); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(producer)); - - testHarness.open(); - - testHarness.processElement(new StreamRecord<>("msg")); - - // make sure that all callbacks have not been completed - verify(mockProducer, times(1)).send(any(ProducerRecord.class), any(Callback.class)); - - // should return even if there are pending records - testHarness.snapshot(123L, 123L); - - testHarness.close(); - } - - // ------------------------------------------------------------------------ - - private static class DummyFlinkKafkaProducer extends FlinkKafkaProducerBase { - private static final long serialVersionUID = 1L; - - private static final String DUMMY_TOPIC = "dummy-topic"; - - private transient KafkaProducer mockProducer; - private transient List pendingCallbacks; - private transient MultiShotLatch flushLatch; - private boolean isFlushed; - - @SuppressWarnings("unchecked") - DummyFlinkKafkaProducer( - Properties producerConfig, - KeyedSerializationSchema schema, - FlinkKafkaPartitioner partitioner) { - - super(DUMMY_TOPIC, schema, producerConfig, partitioner); - - this.mockProducer = mock(KafkaProducer.class); - when(mockProducer.send(any(ProducerRecord.class), any(Callback.class))) - .thenAnswer( - new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) - throws Throwable { - pendingCallbacks.add(invocationOnMock.getArgument(1)); - return null; - } - }); - - this.pendingCallbacks = new ArrayList<>(); - this.flushLatch = new MultiShotLatch(); - } - - long getPendingSize() { - if (flushOnCheckpoint) { - return numPendingRecords(); - } else { - // when flushing is disabled, the implementation does not - // maintain the current number of pending records to reduce - // the extra locking overhead required to do so - throw new UnsupportedOperationException( - "getPendingSize not supported when flushing is disabled"); - } - } - - List getPendingCallbacks() { - return pendingCallbacks; - } - - KafkaProducer getMockKafkaProducer() { - return mockProducer; - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - isFlushed = false; - - super.snapshotState(ctx); - - // if the snapshot implementation doesn't wait until all pending records are flushed, we - // should fail the test - if (flushOnCheckpoint && !isFlushed) { - throw new RuntimeException( - "Flushing is enabled; snapshots should be blocked until all pending records are flushed"); - } - } - - public void waitUntilFlushStarted() throws Exception { - flushLatch.await(); - } - - @SuppressWarnings("unchecked") - @Override - protected KafkaProducer getKafkaProducer(Properties props) { - return (KafkaProducer) mockProducer; - } - - @Override - protected void flush() { - flushLatch.trigger(); - - // simply wait until the producer's pending records become zero. - // This relies on the fact that the producer's Callback implementation - // and pending records tracking logic is implemented correctly, otherwise - // we will loop forever. - while (numPendingRecords() > 0) { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - throw new RuntimeException("Unable to flush producer, task was interrupted"); - } - } - - isFlushed = true; - } - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 7b345bf7d..8644a796c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -45,6 +45,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.assertj.core.api.Assertions.assertThat; @@ -99,23 +100,17 @@ public void resourceCleanUpAtLeastOnce() throws Exception { public void resourceCleanUp(FlinkKafkaProducer.Semantic semantic) throws Exception { String topic = "flink-kafka-producer-resource-cleanup-" + semantic; - final int allowedEpsilonThreadCountGrow = 50; - - Optional initialActiveThreads = Optional.empty(); - for (int i = 0; i < allowedEpsilonThreadCountGrow * 2; i++) { - try (OneInputStreamOperatorTestHarness testHarness1 = - createTestHarness(topic, 1, 1, 0, semantic)) { - testHarness1.setup(); - testHarness1.open(); - } + try (OneInputStreamOperatorTestHarness testHarness1 = + createTestHarness(topic, 1, 1, 0, semantic)) { + testHarness1.setup(); + testHarness1.open(); + testHarness1.snapshot(1L, 100L); + testHarness1.notifyOfCompletedCheckpoint(1L); - if (initialActiveThreads.isPresent()) { - assertThat(Thread.activeCount()) - .as("active threads count") - .isLessThan(initialActiveThreads.get() + allowedEpsilonThreadCountGrow); - } else { - initialActiveThreads = Optional.of(Thread.activeCount()); - } + // test the leak fixed by FLINK-36441 + testHarness1.getOneInputOperator().finish(); + testHarness1.snapshot(2L, 100L); + testHarness1.notifyOfCompletedCheckpoint(2L); } checkProducerLeak(); } @@ -822,12 +817,4 @@ private boolean isCausedBy(FlinkKafkaErrorCode expectedErrorCode, Throwable ex) } return false; } - - private void checkProducerLeak() { - for (Thread t : Thread.getAllStackTraces().keySet()) { - if (t.getName().contains("kafka-producer-network-thread")) { - fail("Detected producer leak. Thread name: " + t.getName()); - } - } - } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java deleted file mode 100644 index 5e87f04b8..000000000 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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. - */ - -package org.apache.flink.streaming.connectors.kafka; - -import org.apache.flink.FlinkVersion; - -import org.junit.Ignore; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collection; - -/** - * Migration test from FlinkKafkaProducer011 operator. This test depends on the resource generated - * by {@link FlinkKafkaProducer011MigrationTest#writeSnapshot()}. - * - *

      Warning: We need to rename the generated resource based on the file naming pattern specified - * by the {@link #getOperatorSnapshotPath(FlinkVersion)} method then copy the resource to the path - * also specified by the {@link #getOperatorSnapshotPath(FlinkVersion)} method. - */ -public class FlinkKafkaProducerMigrationOperatorTest extends FlinkKafkaProducerMigrationTest { - @Parameterized.Parameters(name = "Migration Savepoint: {0}") - public static Collection parameters() { - return Arrays.asList( - FlinkVersion.v1_8, FlinkVersion.v1_9, FlinkVersion.v1_10, FlinkVersion.v1_11); - } - - public FlinkKafkaProducerMigrationOperatorTest(FlinkVersion testMigrateVersion) { - super(testMigrateVersion); - } - - @Override - public String getOperatorSnapshotPath(FlinkVersion version) { - return "src/test/resources/kafka-0.11-migration-kafka-producer-flink-" - + version - + "-snapshot"; - } - - @Ignore - @Override - public void writeSnapshot() throws Exception { - throw new UnsupportedOperationException(); - } -} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 98ab88296..32c380c70 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -42,7 +42,7 @@ public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase { @Parameterized.Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { - return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); + return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.current()); } public FlinkKafkaProducerMigrationTest(FlinkVersion testMigrateVersion) { @@ -54,7 +54,6 @@ protected Properties createProperties() { Properties properties = new Properties(); properties.putAll(standardProps); properties.putAll(secureProps); - properties.put(ProducerConfig.CLIENT_ID_CONFIG, "producer-client-id"); properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "producer-transaction-id"); properties.put(FlinkKafkaProducer.KEY_DISABLE_METRICS, "true"); return properties; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java index 6fedcc43c..5c2d38038 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.kafka.clients.producer.ProducerRecord; +import org.junit.After; import org.junit.Test; import javax.annotation.Nullable; @@ -33,28 +34,35 @@ import java.util.Optional; import java.util.Properties; +import static org.apache.flink.connector.kafka.testutils.KafkaUtil.checkProducerLeak; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link FlinkKafkaProducer}. */ public class FlinkKafkaProducerTest { + @After + public void checkLeaks() { + checkProducerLeak(); + } + @Test public void testOpenSerializationSchemaProducer() throws Exception { OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema(); FlinkKafkaProducer kafkaProducer = new FlinkKafkaProducer<>("localhost:9092", "test-topic", schema); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(schema.openCalled).isTrue(); + assertThat(schema.openCalled).isTrue(); + } } @Test @@ -69,18 +77,19 @@ public void testOpenKafkaSerializationSchemaProducer() throws Exception { properties, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(schema.openCalled).isTrue(); + assertThat(schema.openCalled).isTrue(); + } } @Test @@ -95,18 +104,19 @@ public void testOpenKafkaCustomPartitioner() throws Exception { properties, Optional.of(partitioner)); - OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), 1, 1, 0, IntSerializer.INSTANCE, - new OperatorID(1, 1)); + new OperatorID(1, 1))) { - testHarness.open(); + testHarness.open(); - assertThat(partitioner.openCalled).isTrue(); + assertThat(partitioner.openCalled).isTrue(); + } } @Test(expected = NullPointerException.class) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java index ddbcf1c94..a5abb5e6d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java @@ -17,13 +17,12 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.connector.kafka.util.JacksonMapperFactory; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.streaming.util.serialization.JSONKeyValueDeserializationSchema; -import org.apache.flink.util.jackson.JacksonMapperFactory; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Test; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index a8596ee7c..9a9acdea0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -69,6 +69,7 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators; import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; +import org.apache.flink.streaming.connectors.kafka.testutils.KafkaUtils; import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper; import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper; import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2FlinkPartitioner; @@ -80,8 +81,6 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; - import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetResetStrategy; @@ -104,6 +103,7 @@ import java.util.Collections; import java.util.Date; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -258,7 +258,10 @@ public void run() { } while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); + final Iterator it = getRunningJobs(client).iterator(); + final JobID jobId = it.next(); + client.cancel(jobId).get(); + assertThat(it.hasNext()).isFalse(); runner.join(); final Throwable t = errorRef.get(); @@ -349,7 +352,10 @@ public void run() { } while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); + final Iterator it = getRunningJobs(client).iterator(); + final JobID jobId = it.next(); + client.cancel(jobId).get(); + assertThat(it.hasNext()).isFalse(); runner.join(); final Throwable t = errorRef.get(); @@ -864,7 +870,7 @@ public void cancel() { } }); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); kafkaServer.produceIntoKafka(stream, topic, sinkSchema, producerProperties, null); @@ -1545,7 +1551,7 @@ public void cancel() {} new TypeInformationKeyValueSerializationSchema<>( Long.class, PojoValue.class, env.getConfig()); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); env.execute("Write KV to Kafka"); @@ -1641,7 +1647,7 @@ public void cancel() {} byte[].class, PojoValue.class, env.getConfig()); Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); producerProperties.putAll(secureProps); kafkaServer.produceIntoKafka(kvStream, topic, schema, producerProperties, null); @@ -2091,7 +2097,7 @@ public void flatMap( count++; - LOG.info("Received message {}, total {} messages", value, count); + LOG.debug("Received message {}, total {} messages", value, count); // verify if we've seen everything if (count == finalCount) { @@ -2283,7 +2289,7 @@ public void cancel() { // the producer must not produce duplicates Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); @@ -2387,7 +2393,7 @@ public void cancel() { // the producer must not produce duplicates Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); producerProperties.putAll(secureProps); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 624381068..cf3bf463f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -36,6 +36,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; import org.apache.flink.streaming.connectors.kafka.testutils.IntegerSource; +import org.apache.flink.streaming.connectors.kafka.testutils.KafkaUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.Preconditions; @@ -148,8 +149,7 @@ public void cancel() { .setParallelism(1); Properties props = new Properties(); - props.putAll( - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings)); + props.putAll(KafkaUtils.getPropertiesFromBrokerList(brokerConnectionStrings)); props.putAll(secureProps); // sink partitions into diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java index 0d89d39b8..90ce2e5eb 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaSerializerUpgradeTest.java @@ -20,10 +20,10 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerMatchers; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase; import org.apache.flink.streaming.connectors.kafka.internals.FlinkKafkaInternalProducer; +import org.apache.flink.streaming.connectors.kafka.testutils.TypeSerializerMatchers; +import org.apache.flink.streaming.connectors.kafka.testutils.TypeSerializerUpgradeTestBase; import org.hamcrest.Matcher; import org.mockito.Mockito; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index b4fd54ab2..eb35391f5 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -32,6 +32,7 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; +import com.google.common.base.MoreObjects; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -45,6 +46,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; @@ -52,11 +55,8 @@ import java.util.List; import java.util.Properties; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import scala.concurrent.duration.FiniteDuration; - import static org.assertj.core.api.Assertions.fail; /** @@ -83,15 +83,16 @@ public abstract class KafkaTestBase extends TestLogger { public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); public static final int NUMBER_OF_KAFKA_SERVERS = 1; + private static int numKafkaClusters = 1; public static String brokerConnectionStrings; public static Properties standardProps; - public static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); - public static KafkaTestEnvironment kafkaServer; + public static List kafkaClusters = new ArrayList<>(); + @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); public static Properties secureProps = new Properties(); @@ -108,7 +109,7 @@ public static void prepare() throws Exception { LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); - startClusters(false); + startClusters(false, numKafkaClusters); } @AfterClass @@ -140,18 +141,28 @@ public static void startClusters() throws Exception { KafkaTestEnvironment.createConfig().setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS)); } - public static void startClusters(boolean secureMode) throws Exception { + public static void startClusters(boolean secureMode, int numKafkaClusters) throws Exception { startClusters( - KafkaTestEnvironment.createConfig() - .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS) - .setSecureMode(secureMode)); + KafkaTestEnvironment.createConfig().setSecureMode(secureMode), numKafkaClusters); + } + + public static void startClusters( + KafkaTestEnvironment.Config environmentConfig, int numKafkaClusters) throws Exception { + for (int i = 0; i < numKafkaClusters; i++) { + startClusters(environmentConfig); + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + new KafkaClusterTestEnvMetadata( + i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); + kafkaClusters.add(kafkaClusterTestEnvMetadata); + LOG.info("Created Kafka cluster with configuration: {}", kafkaClusterTestEnvMetadata); + } } public static void startClusters(KafkaTestEnvironment.Config environmentConfig) throws Exception { kafkaServer = constructKafkaTestEnvironment(); - LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); + LOG.info("Starting KafkaTestBase.prepare() for Kafka {}", kafkaServer.getVersion()); kafkaServer.prepare(environmentConfig); @@ -183,6 +194,13 @@ public static void shutdownClusters() throws Exception { if (kafkaServer != null) { kafkaServer.shutdown(); } + + if (kafkaClusters != null && !kafkaClusters.isEmpty()) { + for (KafkaClusterTestEnvMetadata value : kafkaClusters) { + value.getKafkaTestEnvironment().shutdown(); + } + kafkaClusters.clear(); + } } // ------------------------------------------------------------------------ @@ -221,11 +239,15 @@ public static void produceToKafka( Collection> records, Class> keySerializerClass, Class> - valueSerializerClass) + valueSerializerClass, + @Nullable Properties extraProps) throws Throwable { Properties props = new Properties(); props.putAll(standardProps); props.putAll(kafkaServer.getIdempotentProducerConfig()); + if (extraProps != null) { + props.putAll(extraProps); + } props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass.getName()); props.setProperty( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); @@ -240,9 +262,16 @@ public static void produceToKafka( } }; try (KafkaProducer producer = new KafkaProducer<>(props)) { + if (props.containsKey(ProducerConfig.TRANSACTIONAL_ID_CONFIG)) { + producer.initTransactions(); + producer.beginTransaction(); + } for (ProducerRecord record : records) { producer.send(record, callback); } + if (props.containsKey(ProducerConfig.TRANSACTIONAL_ID_CONFIG)) { + producer.commitTransaction(); + } } if (sendingError.get() != null) { throw sendingError.get(); @@ -338,4 +367,62 @@ private String formatElements(List elements) { return String.format("elements: <%s>", elements); } } + + public static void setNumKafkaClusters(int size) { + numKafkaClusters = size; + } + + /** Metadata generated by this test utility. */ + public static class KafkaClusterTestEnvMetadata { + + private final String kafkaClusterId; + private final KafkaTestEnvironment kafkaTestEnvironment; + private final Properties standardProperties; + private final String brokerConnectionStrings; + private final Properties secureProperties; + + private KafkaClusterTestEnvMetadata( + int kafkaClusterIdx, + KafkaTestEnvironment kafkaTestEnvironment, + Properties standardProperties, + String brokerConnectionStrings, + Properties secureProperties) { + this.kafkaClusterId = "kafka-cluster-" + kafkaClusterIdx; + this.kafkaTestEnvironment = kafkaTestEnvironment; + this.standardProperties = standardProperties; + this.brokerConnectionStrings = brokerConnectionStrings; + this.secureProperties = secureProperties; + } + + public String getKafkaClusterId() { + return kafkaClusterId; + } + + public KafkaTestEnvironment getKafkaTestEnvironment() { + return kafkaTestEnvironment; + } + + public Properties getStandardProperties() { + return standardProperties; + } + + public String getBrokerConnectionStrings() { + return brokerConnectionStrings; + } + + public Properties getSecureProperties() { + return secureProperties; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("kafkaClusterId", kafkaClusterId) + .add("kafkaTestEnvironment", kafkaTestEnvironment) + .add("standardProperties", standardProperties) + .add("brokerConnectionStrings", brokerConnectionStrings) + .add("secureProperties", secureProperties) + .toString(); + } + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index ee38e8501..1494ff1f4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -34,11 +34,14 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.UUID; /** Abstract class providing a Kafka test environment. */ public abstract class KafkaTestEnvironment { /** Configuration class for {@link KafkaTestEnvironment}. */ public static class Config { + + private int numKafkaClusters = 1; private int kafkaServersNumber = 1; private Properties kafkaServerProperties = null; private boolean secureMode = false; @@ -113,6 +116,12 @@ public Properties getIdempotentProducerConfig() { return props; } + public Properties getTransactionalProducerConfig() { + Properties props = new Properties(); + props.put("transactional.id", UUID.randomUUID().toString()); + return props; + } + // -- consumer / producer instances: public FlinkKafkaConsumerBase getConsumer( List topics, DeserializationSchema deserializationSchema, Properties props) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index c5bc3b003..31ddbbedf 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.kafka.source.KafkaSource; import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.streaming.api.datastream.DataStream; @@ -28,7 +29,6 @@ import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.apache.flink.util.DockerImageVersions; import org.apache.commons.collections.list.UnmodifiableList; import org.apache.kafka.clients.admin.AdminClient; @@ -169,8 +169,13 @@ private void tryDelete(AdminClient adminClient, String topic) throws Exception { @Override public void createTestTopic( String topic, int numberOfPartitions, int replicationFactor, Properties properties) { + createNewTopic(topic, numberOfPartitions, replicationFactor, getStandardProperties()); + } + + public static void createNewTopic( + String topic, int numberOfPartitions, int replicationFactor, Properties properties) { LOG.info("Creating topic {}", topic); - try (AdminClient adminClient = AdminClient.create(getStandardProperties())) { + try (AdminClient adminClient = AdminClient.create(properties)) { NewTopic topicObj = new NewTopic(topic, numberOfPartitions, (short) replicationFactor); adminClient.createTopics(Collections.singleton(topicObj)).all().get(); CommonTestUtils.waitUtil( @@ -429,7 +434,7 @@ private KafkaContainer createKafkaContainer( int brokerID, @Nullable GenericContainer zookeeper) { String brokerName = String.format("Kafka-%d", brokerID); KafkaContainer broker = - KafkaUtil.createKafkaContainer(DockerImageVersions.KAFKA, LOG, brokerName) + KafkaUtil.createKafkaContainer(brokerName) .withNetworkAliases(brokerName) .withEnv("KAFKA_BROKER_ID", String.valueOf(brokerID)) .withEnv("KAFKA_MESSAGE_MAX_BYTES", String.valueOf(50 * 1024 * 1024)) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java index 269a1552b..295db6155 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/metrics/KafkaMetricMutableWrapperTest.java @@ -28,8 +28,6 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; import org.testcontainers.containers.Network; import org.testcontainers.junit.jupiter.Container; @@ -43,19 +41,17 @@ import java.util.stream.Stream; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; -import static org.apache.flink.util.DockerImageVersions.KAFKA; @Testcontainers @ExtendWith(TestLoggerExtension.class) class KafkaMetricMutableWrapperTest { - private static final Logger LOG = LoggerFactory.getLogger(KafkaMetricMutableWrapperTest.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final Network NETWORK = Network.newNetwork(); @Container public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) + createKafkaContainer(KafkaMetricMutableWrapperTest.class) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java index 9a0a14da3..5505bdde3 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java @@ -34,9 +34,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaShuffleFetcher.KafkaShuffleWatermark; import org.apache.flink.util.PropertiesUtil; -import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Rule; import org.junit.Test; @@ -374,17 +371,16 @@ private void testRecordSerDe(TimeCharacteristic timeCharacteristic) throws Excep // Records in a single partition are kept in order Collection> records = - Iterables.getOnlyElement( - testKafkaShuffleProducer( - topic( - "test_serde-" + UUID.randomUUID(), - timeCharacteristic), - env, - 1, - 1, - numElementsPerProducer, - timeCharacteristic) - .values()); + testKafkaShuffleProducer( + topic("test_serde-" + UUID.randomUUID(), timeCharacteristic), + env, + 1, + 1, + numElementsPerProducer, + timeCharacteristic) + .values() + .iterator() + .next(); switch (timeCharacteristic) { case ProcessingTime: @@ -516,7 +512,7 @@ private Map>> testKafkaShuffl r -> { final int partition = r.partition(); if (!results.containsKey(partition)) { - results.put(partition, Lists.newArrayList()); + results.put(partition, new ArrayList<>()); } results.get(partition).add(r); }); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java new file mode 100644 index 000000000..6371ae5ef --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchemaTest.java @@ -0,0 +1,148 @@ +package org.apache.flink.streaming.connectors.kafka.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.util.TestLogger; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link DynamicKafkaRecordSerializationSchema}. */ +public class DynamicKafkaRecordSerializationSchemaTest extends TestLogger { + private static final List MULTIPLE_TOPICS = Arrays.asList("topic1", "topic2"); + private static final String SINGLE_TOPIC = "topic"; + private static final Pattern TOPIC_PATTERN = Pattern.compile("topic*"); + + @ParameterizedTest + @MethodSource("provideTopicMetadataTestParameters") + public void testTopicMetadata( + List topics, Pattern topicPattern, String rowTopic, String expectedTopic) { + GenericRowData rowData = createRowData(rowTopic); + DynamicKafkaRecordSerializationSchema schema = createSchema(topics, topicPattern); + KafkaRecordSerializationSchema.KafkaSinkContext context = createContext(); + + // Call serialize method + ProducerRecord record = schema.serialize(rowData, context, null); + + // Assert the returned ProducerRecord is routed to the correct topic + assertThat(record.topic()).isEqualTo(expectedTopic); + } + + @ParameterizedTest + @MethodSource("provideInvalidTopicMetadataTestParameters") + public void testInvalidTopicMetadata( + List topics, Pattern topicPattern, String rowTopic, String expectedError) { + GenericRowData rowData = createRowData(rowTopic); + DynamicKafkaRecordSerializationSchema schema = createSchema(topics, topicPattern); + KafkaRecordSerializationSchema.KafkaSinkContext context = createContext(); + + // Call serialize method + assertThatThrownBy(() -> schema.serialize(rowData, context, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(expectedError); + } + + private static Stream provideTopicMetadataTestParameters() { + String topic1 = "topic1"; + return Stream.of( + Arguments.of( + Collections.singletonList(SINGLE_TOPIC), null, SINGLE_TOPIC, SINGLE_TOPIC), + Arguments.of(Collections.singletonList(SINGLE_TOPIC), null, topic1, SINGLE_TOPIC), + Arguments.of(Collections.singletonList(SINGLE_TOPIC), null, null, SINGLE_TOPIC), + Arguments.of(MULTIPLE_TOPICS, null, topic1, topic1), + Arguments.of(null, TOPIC_PATTERN, SINGLE_TOPIC, SINGLE_TOPIC)); + } + + private static Stream provideInvalidTopicMetadataTestParameters() { + String other = "other"; + return Stream.of( + Arguments.of( + MULTIPLE_TOPICS, + null, + other, + String.format( + "The topic of the sink record is not valid. Expected topic to be in: %s but was: %s", + MULTIPLE_TOPICS, other)), + Arguments.of( + null, + TOPIC_PATTERN, + other, + String.format( + "The topic of the sink record is not valid. Expected topic to match: %s but was: %s", + "topic*", other))); + } + + private DynamicKafkaRecordSerializationSchema createSchema( + List topics, Pattern topicPattern) { + // Create a SerializationSchema for RowData + SerializationSchema serializationSchema = + new SerializationSchema() { + @Override + public byte[] serialize(RowData element) { + return ((StringData) element.getString(0)).toBytes(); + } + + @Override + public void open(InitializationContext context) throws Exception {} + }; + + int[] metadataPositions = new int[3]; + metadataPositions[KafkaDynamicSink.WritableMetadata.TOPIC.ordinal()] = 1; + metadataPositions[KafkaDynamicSink.WritableMetadata.HEADERS.ordinal()] = 2; + metadataPositions[KafkaDynamicSink.WritableMetadata.TIMESTAMP.ordinal()] = 3; + + return new DynamicKafkaRecordSerializationSchema( + topics, + topicPattern, + null, + null, + serializationSchema, + new RowData.FieldGetter[] {r -> r.getString(0)}, + new RowData.FieldGetter[] {r -> r.getString(0)}, + true, + metadataPositions, + false); + } + + private GenericRowData createRowData(String topic) { + GenericRowData rowData = new GenericRowData(4); + rowData.setField(0, StringData.fromString("test")); + rowData.setField(1, StringData.fromString(topic)); + rowData.setField(2, null); + rowData.setField(3, null); + return rowData; + } + + private KafkaRecordSerializationSchema.KafkaSinkContext createContext() { + return new KafkaRecordSerializationSchema.KafkaSinkContext() { + @Override + public int getParallelInstanceId() { + return 0; + } + + @Override + public int getNumberOfParallelInstances() { + return 1; + } + + @Override + public int[] getPartitionsForTopic(String topic) { + return new int[] {0}; + } + }; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java index e8bc9e373..cd8e58a79 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java @@ -29,7 +29,6 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.Before; @@ -65,7 +64,6 @@ public void testKafkaDebeziumChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Debezium json into Kafka ------------------- List lines = readLines("debezium-data-schema-exclude.txt"); @@ -194,7 +192,6 @@ public void testKafkaCanalChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Canal json into Kafka ------------------- List lines = readLines("canal-data.txt"); @@ -335,7 +332,6 @@ public void testKafkaMaxwellChangelogSource() throws Exception { tableConf.set( ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)); tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L); - tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); // ---------- Write the Maxwell json into Kafka ------------------- List lines = readLines("maxwell-data.txt"); @@ -478,6 +474,7 @@ private void writeRecordsToKafka(String topic, List lines) throws Except .setPartitioner(partitioner) .build()) .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .setTransactionalIdPrefix("kafka-sink") .build()); env.execute("Write sequence"); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index a0b74a5b7..10bfe5939 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -46,6 +46,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; +import org.apache.flink.streaming.connectors.kafka.testutils.MockPartitionOffsetsRetriever; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; @@ -73,8 +74,6 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; - import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; @@ -87,7 +86,6 @@ import javax.annotation.Nullable; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -97,13 +95,13 @@ import java.util.Properties; import java.util.Set; import java.util.function.Consumer; -import java.util.function.Function; import java.util.regex.Pattern; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.DEBEZIUM_AVRO_CONFLUENT; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.flink.table.factories.FactoryUtil.SOURCE_PARALLELISM; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; @@ -215,7 +213,8 @@ public void testTableSource() { KAFKA_SOURCE_PROPERTIES, StartupMode.SPECIFIC_OFFSETS, specificOffsets, - 0); + 0, + null); assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); ScanTableSource.ScanRuntimeProvider provider = @@ -223,6 +222,48 @@ public void testTableSource() { assertKafkaSource(provider); } + @Test + public void testTableSourceWithParallelism() { + final Map modifiedOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.put(SOURCE_PARALLELISM.key(), "100")); + final DynamicTableSource actualSource = createTableSource(SCHEMA, modifiedOptions); + final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; + + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + KAFKA_SOURCE_PROPERTIES, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0, + 100); + assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final DataStreamScanProvider sourceProvider = (DataStreamScanProvider) provider; + assertThat(sourceProvider.getParallelism()).isPresent(); + assertThat(sourceProvider.getParallelism()).hasValue(100); + } + @Test public void testTableSourceWithPattern() { final Map modifiedOptions = @@ -257,7 +298,8 @@ public void testTableSourceWithPattern() { KAFKA_SOURCE_PROPERTIES, StartupMode.EARLIEST, specificOffsets, - 0); + 0, + null); final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; assertThat(actualKafkaSource).isEqualTo(expectedKafkaSource); @@ -298,7 +340,8 @@ public void testTableSourceWithKeyValue() { KAFKA_FINAL_SOURCE_PROPERTIES, StartupMode.GROUP_OFFSETS, Collections.emptyMap(), - 0); + 0, + null); assertThat(actualSource).isEqualTo(expectedKafkaSource); } @@ -349,7 +392,8 @@ public void testTableSourceWithKeyValueAndMetadata() { KAFKA_FINAL_SOURCE_PROPERTIES, StartupMode.GROUP_OFFSETS, Collections.emptyMap(), - 0); + 0, + null); expectedKafkaSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedKafkaSource.metadataKeys = Collections.singletonList("timestamp"); @@ -480,13 +524,17 @@ public void testBoundedLatestOffset() { OffsetsInitializer offsetsInitializer = KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); TopicPartition partition = new TopicPartition(TOPIC, 0); + long endOffsets = 123L; Map partitionOffsets = offsetsInitializer.getPartitionOffsets( Collections.singletonList(partition), - MockPartitionOffsetsRetriever.noInteractions()); + MockPartitionOffsetsRetriever.latest( + (tps) -> + Collections.singletonMap( + partition, endOffsets))); assertThat(partitionOffsets) .containsOnlyKeys(partition) - .containsEntry(partition, KafkaPartitionSplit.LATEST_OFFSET); + .containsEntry(partition, endOffsets); }); } @@ -577,75 +625,6 @@ private void testBoundedOffsets( validator.accept(kafkaSource); } - private interface OffsetsRetriever - extends Function, Map> {} - - private interface TimestampOffsetsRetriever - extends Function, Map> {} - - private static final class MockPartitionOffsetsRetriever - implements OffsetsInitializer.PartitionOffsetsRetriever { - - public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }; - private final OffsetsRetriever committedOffsets; - private final OffsetsRetriever endOffsets; - private final OffsetsRetriever beginningOffsets; - private final TimestampOffsetsRetriever offsetsForTimes; - - static MockPartitionOffsetsRetriever noInteractions() { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - UNSUPPORTED_RETRIEVAL, - partitions -> { - throw new UnsupportedOperationException( - "The method was not supposed to be called"); - }); - } - - static MockPartitionOffsetsRetriever timestampAndEnd( - TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { - return new MockPartitionOffsetsRetriever( - UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); - } - - private MockPartitionOffsetsRetriever( - OffsetsRetriever committedOffsets, - OffsetsRetriever endOffsets, - OffsetsRetriever beginningOffsets, - TimestampOffsetsRetriever offsetsForTimes) { - this.committedOffsets = committedOffsets; - this.endOffsets = endOffsets; - this.beginningOffsets = beginningOffsets; - this.offsetsForTimes = offsetsForTimes; - } - - @Override - public Map committedOffsets(Collection partitions) { - return committedOffsets.apply(partitions); - } - - @Override - public Map endOffsets(Collection partitions) { - return endOffsets.apply(partitions); - } - - @Override - public Map beginningOffsets(Collection partitions) { - return beginningOffsets.apply(partitions); - } - - @Override - public Map offsetsForTimes( - Map timestampsToSearch) { - return offsetsForTimes.apply(timestampsToSearch); - } - } - @Test public void testTableSink() { final Map modifiedOptions = @@ -668,7 +647,8 @@ public void testTableSink() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -684,11 +664,15 @@ public void testTableSink() { final SinkV2Provider sinkProvider = (SinkV2Provider) provider; final Sink sinkFunction = sinkProvider.createSink(); assertThat(sinkFunction).isInstanceOf(KafkaSink.class); + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); } @Test public void testTableSinkSemanticTranslation() { - final List semantics = ImmutableList.of("exactly-once", "at-least-once", "none"); + final List semantics = Arrays.asList("exactly-once", "at-least-once", "none"); final EncodingFormat> valueEncodingFormat = new EncodingFormatMock(","); for (final String semantic : semantics) { @@ -708,7 +692,8 @@ public void testTableSinkSemanticTranslation() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.valueOf(semantic.toUpperCase().replace("-", "_")), @@ -751,7 +736,8 @@ public void testTableSinkWithKeyValue() { new int[] {0}, new int[] {1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_FINAL_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -779,7 +765,8 @@ public void testTableSinkWithParallelism() { new int[0], new int[] {0, 1, 2}, null, - TOPIC, + Collections.singletonList(TOPIC), + null, KAFKA_SINK_PROPERTIES, new FlinkFixedPartitioner<>(), DeliveryGuarantee.EXACTLY_ONCE, @@ -873,6 +860,77 @@ public void testTableSinkAutoCompleteSchemaRegistrySubject() { "sub2"); } + @Test + public void testTableSinkWithTopicList() { + final Map modifiedOptions = + getModifiedOptions(getBasicSinkOptions(), options -> options.put("topic", TOPICS)); + KafkaDynamicSink actualSink = (KafkaDynamicSink) createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new EncodingFormatMock(","); + + final DynamicTableSink expectedSink = + createExpectedSink( + SCHEMA_DATA_TYPE, + null, + valueEncodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Arrays.asList(TOPICS.split(";")), + null, + KAFKA_SINK_PROPERTIES, + new FlinkFixedPartitioner<>(), + DeliveryGuarantee.EXACTLY_ONCE, + null, + "kafka-sink"); + assertThat(actualSink).isEqualTo(expectedSink); + final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.TOPIC.key, + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); + } + + @Test + public void testTableSinkWithTopicPattern() { + final Map modifiedOptions = + getModifiedOptions( + getBasicSinkOptions(), + options -> { + options.remove("topic"); + options.put("topic-pattern", TOPIC_REGEX); + }); + KafkaDynamicSink actualSink = (KafkaDynamicSink) createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new EncodingFormatMock(","); + + final DynamicTableSink expectedSink = + createExpectedSink( + SCHEMA_DATA_TYPE, + null, + valueEncodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + null, + Pattern.compile(TOPIC_REGEX), + KAFKA_SINK_PROPERTIES, + new FlinkFixedPartitioner<>(), + DeliveryGuarantee.EXACTLY_ONCE, + null, + "kafka-sink"); + assertThat(actualSink).isEqualTo(expectedSink); + final KafkaDynamicSink actualKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualKafkaSink.listWritableMetadata()) + .containsOnlyKeys( + KafkaDynamicSink.WritableMetadata.TOPIC.key, + KafkaDynamicSink.WritableMetadata.HEADERS.key, + KafkaDynamicSink.WritableMetadata.TIMESTAMP.key); + } + private void verifyEncoderSubject( Consumer> optionModifier, String expectedValueSubject, @@ -1070,7 +1128,7 @@ public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { } @Test - public void testSinkWithTopicListOrTopicPattern() { + public void testSinkWithTopicListAndTopicPattern() { Map modifiedOptions = getModifiedOptions( getBasicSinkOptions(), @@ -1078,32 +1136,13 @@ public void testSinkWithTopicListOrTopicPattern() { options.put("topic", TOPICS); options.put("scan.startup.mode", "earliest-offset"); options.remove("specific-offsets"); + options.put("topic-pattern", TOPIC_REGEX); }); final String errorMessageTemp = - "Flink Kafka sink currently only supports single topic, but got %s: %s."; - - try { - createTableSink(SCHEMA, modifiedOptions); - } catch (Throwable t) { - assertThat(t.getCause().getMessage()) - .isEqualTo( - String.format( - errorMessageTemp, - "'topic'", - String.format("[%s]", String.join(", ", TOPIC_LIST)))); - } - - modifiedOptions = - getModifiedOptions( - getBasicSinkOptions(), - options -> options.put("topic-pattern", TOPIC_REGEX)); - - try { - createTableSink(SCHEMA, modifiedOptions); - } catch (Throwable t) { - assertThat(t.getCause().getMessage()) - .isEqualTo(String.format(errorMessageTemp, "'topic-pattern'", TOPIC_REGEX)); - } + "Option 'topic' and 'topic-pattern' shouldn't be set together."; + assertThatThrownBy(() -> createTableSink(SCHEMA, modifiedOptions)) + .isInstanceOf(ValidationException.class) + .satisfies(anyCauseMatches(ValidationException.class, errorMessageTemp)); } @Test @@ -1165,6 +1204,84 @@ public void testPrimaryKeyValidation() { + " guarantee the semantic of primary key."); } + @Test + public void testDiscoverPartitionByDefault() { + Map tableSourceOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.remove("scan.topic-partition-discovery.interval")); + final KafkaDynamicSource actualSource = + (KafkaDynamicSource) createTableSource(SCHEMA, tableSourceOptions); + Properties props = new Properties(); + props.putAll(KAFKA_SOURCE_PROPERTIES); + // The default partition discovery interval is 5 minutes + props.setProperty("partition.discovery.interval.ms", "300000"); + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + props, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0, + null); + assertThat(actualSource).isEqualTo(expectedKafkaSource); + ScanTableSource.ScanRuntimeProvider provider = + actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + + @Test + public void testDisableDiscoverPartition() { + Map tableSourceOptions = + getModifiedOptions( + getBasicSourceOptions(), + options -> options.put("scan.topic-partition-discovery.interval", "0")); + final KafkaDynamicSource actualSource = + (KafkaDynamicSource) createTableSource(SCHEMA, tableSourceOptions); + Properties props = new Properties(); + props.putAll(KAFKA_SOURCE_PROPERTIES); + // Disable discovery if the partition discovery interval is 0 minutes + props.setProperty("partition.discovery.interval.ms", "0"); + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + final DecodingFormat> valueDecodingFormat = + new DecodingFormatMock(",", true); + // Test scan source equals + final KafkaDynamicSource expectedKafkaSource = + createExpectedScanSource( + SCHEMA_DATA_TYPE, + null, + valueDecodingFormat, + new int[0], + new int[] {0, 1, 2}, + null, + Collections.singletonList(TOPIC), + null, + props, + StartupMode.SPECIFIC_OFFSETS, + specificOffsets, + 0, + null); + assertThat(actualSource).isEqualTo(expectedKafkaSource); + ScanTableSource.ScanRuntimeProvider provider = + actualSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @@ -1181,7 +1298,8 @@ private static KafkaDynamicSource createExpectedScanSource( Properties properties, StartupMode startupMode, Map specificStartupOffsets, - long startupTimestampMillis) { + long startupTimestampMillis, + @Nullable Integer parallelism) { return new KafkaDynamicSource( physicalDataType, keyDecodingFormat, @@ -1199,7 +1317,8 @@ private static KafkaDynamicSource createExpectedScanSource( Collections.emptyMap(), 0, false, - FactoryMocks.IDENTIFIER.asSummaryString()); + FactoryMocks.IDENTIFIER.asSummaryString(), + parallelism); } private static KafkaDynamicSink createExpectedSink( @@ -1209,7 +1328,8 @@ private static KafkaDynamicSink createExpectedSink( int[] keyProjection, int[] valueProjection, @Nullable String keyPrefix, - String topic, + @Nullable List topics, + @Nullable Pattern topicPattern, Properties properties, @Nullable FlinkKafkaPartitioner partitioner, DeliveryGuarantee deliveryGuarantee, @@ -1223,7 +1343,8 @@ private static KafkaDynamicSink createExpectedSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, partitioner, deliveryGuarantee, diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 515526f93..acd0550e4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -18,23 +18,32 @@ package org.apache.flink.streaming.connectors.kafka.table; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.messages.FlinkJobTerminatedWithoutCancellationException; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.function.RunnableWithException; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.assertj.core.api.Assertions; +import org.assertj.core.api.ThrowingConsumer; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -48,6 +57,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -58,6 +68,8 @@ import java.util.stream.IntStream; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectAllRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT; @@ -178,7 +190,227 @@ public void testKafkaSourceSink() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); + } + + @Test + public void testKafkaSourceSinkWithTopicList() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic1 = "topics1_" + format + "_" + UUID.randomUUID(); + final String topic2 = "topics2_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic2, 1, 1); + createTestTopic(topic1, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + final String createTableTemplate = + "CREATE TABLE %s (\n" + + " `topic` STRING METADATA,\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'earliest-offset',\n" + + " 'scan.bounded.mode' = 'latest-offset',\n" + + " %s\n" + + ")\n"; + final String createTopicListTable = + String.format( + createTableTemplate, + "kafka", + KafkaDynamicTableFactory.IDENTIFIER, + String.join(";", Arrays.asList(topic1, topic2)), + bootstraps, + groupId, + formatOptions()); + final String createTopic1Table = + String.format( + createTableTemplate, + "topic1", + KafkaDynamicTableFactory.IDENTIFIER, + topic1, + bootstraps, + groupId, + formatOptions()); + final String createTopic2Table = + String.format( + createTableTemplate, + "topic2", + KafkaDynamicTableFactory.IDENTIFIER, + topic2, + bootstraps, + groupId, + formatOptions()); + + tEnv.executeSql(createTopicListTable); + tEnv.executeSql(createTopic1Table); + tEnv.executeSql(createTopic2Table); + + List values = + Arrays.asList( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + + // ---------- Consume stream from Kafka ------------------- + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); + List topic1Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic1")); + List topic2Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic2")); + assertThat(results) + .containsExactlyInAnyOrder( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + assertThat(topic1Results).containsExactly(Row.of(topic1, 1, 1102, "behavior 1")); + assertThat(topic2Results).containsExactly(Row.of(topic2, 2, 1103, "behavior 2")); + + // ------------- cleanup ------------------- + cleanupTopic(topic1); + cleanupTopic(topic2); + } + + @Test + public void testKafkaSourceSinkWithTopicPattern() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic1 = "topics1_" + format + "_" + UUID.randomUUID(); + final String topic2 = "topics2_" + format + "_" + UUID.randomUUID(); + final String topicPattern = "topics.*"; + createTestTopic(topic2, 1, 1); + createTestTopic(topic1, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + final String createTableTemplate = + "CREATE TABLE %s (\n" + + " `topic` STRING METADATA,\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic-pattern' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'earliest-offset',\n" + + " 'scan.bounded.mode' = 'latest-offset',\n" + + " %s\n" + + ")\n"; + final String createTopicPatternTable = + String.format( + createTableTemplate, + "kafka", + KafkaDynamicTableFactory.IDENTIFIER, + topicPattern, + bootstraps, + groupId, + formatOptions()); + final String createTopic1Table = + String.format( + createTableTemplate, + "topic1", + KafkaDynamicTableFactory.IDENTIFIER, + topic1, + bootstraps, + groupId, + formatOptions()); + final String createTopic2Table = + String.format( + createTableTemplate, + "topic2", + KafkaDynamicTableFactory.IDENTIFIER, + topic2, + bootstraps, + groupId, + formatOptions()); + + tEnv.executeSql(createTopicPatternTable); + tEnv.executeSql(createTopic1Table); + tEnv.executeSql(createTopic2Table); + + List values = + Arrays.asList( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + + // ---------- Consume stream from Kafka ------------------- + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); + List topic1Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic1")); + List topic2Results = collectAllRows(tEnv.sqlQuery("SELECT * from topic2")); + assertThat(results) + .containsExactlyInAnyOrder( + Row.of(topic1, 1, 1102, "behavior 1"), + Row.of(topic2, 2, 1103, "behavior 2")); + assertThat(topic1Results).containsExactly(Row.of(topic1, 1, 1102, "behavior 1")); + assertThat(topic2Results).containsExactly(Row.of(topic2, 2, 1103, "behavior 2")); + + // ------------- cleanup ------------------- + + cleanupTopic(topic1); + cleanupTopic(topic2); + } + + @Test + public void testKafkaSourceEmptyResultOnDeletedOffsets() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "bounded_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + // ---------- Produce an event time stream into Kafka ------------------- + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + + final String createTable = + String.format( + "CREATE TABLE kafka (\n" + + " `user_id` INT,\n" + + " `item_id` INT,\n" + + " `behavior` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'specific-offsets',\n" + + " 'scan.bounded.mode' = 'specific-offsets',\n" + + " 'scan.startup.specific-offsets' = 'partition:0,offset:1',\n" + + " 'scan.bounded.specific-offsets' = 'partition:0,offset:3',\n" + + " %s\n" + + ")\n", + KafkaDynamicTableFactory.IDENTIFIER, + topic, + bootstraps, + groupId, + formatOptions()); + tEnv.executeSql(createTable); + List values = + Arrays.asList( + Row.of(1, 1102, "behavior 1"), + Row.of(2, 1103, "behavior 2"), + Row.of(3, 1104, "behavior 3")); + tEnv.fromValues(values).insertInto("kafka").execute().await(); + // ---------- Delete events from Kafka ------------------- + Map partitionOffsetsToDelete = new HashMap<>(); + partitionOffsetsToDelete.put(0, 3L); + deleteRecords(topic, partitionOffsetsToDelete); + // ---------- Consume stream from Kafka ------------------- + List results = new ArrayList<>(); + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + tEnv.executeSql(createTable); + results.addAll(collectAllRows(tEnv.sqlQuery("SELECT * FROM kafka"))); + assertThat(results).isEmpty(); + + // ------------- cleanup ------------------- + + cleanupTopic(topic); } @Test @@ -225,20 +457,14 @@ public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // ---------- Consume stream from Kafka ------------------- - List results = new ArrayList<>(); - try (CloseableIterator resultsItr = - tEnv.sqlQuery("SELECT * from kafka").execute().collect()) { - while (resultsItr.hasNext()) { - results.add(resultsItr.next()); - } - } + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); assertThat(results) .containsExactly(Row.of(1, 1102, "behavior 1"), Row.of(2, 1103, "behavior 2")); // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -286,13 +512,7 @@ public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { // ---------- Consume stream from Kafka ------------------- - List results = new ArrayList<>(); - try (CloseableIterator resultsItr = - tEnv.sqlQuery("SELECT * from kafka").execute().collect()) { - while (resultsItr.hasNext()) { - results.add(resultsItr.next()); - } - } + List results = collectAllRows(tEnv.sqlQuery("SELECT * from kafka")); assertThat(results) .containsExactly( @@ -301,7 +521,7 @@ public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -486,7 +706,7 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -567,7 +787,7 @@ public void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -645,7 +865,7 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(topic); + cleanupTopic(topic); } @Test @@ -761,8 +981,8 @@ public void testKafkaTemporalJoinChangelog() throws Exception { // ------------- cleanup ------------------- - deleteTestTopic(orderTopic); - deleteTestTopic(productTopic); + cleanupTopic(orderTopic); + cleanupTopic(productTopic); } private void initialProductChangelog(String topic, String bootstraps) throws Exception { @@ -877,8 +1097,8 @@ public void testPerPartitionWatermarkKafka() throws Exception { // ------------- cleanup ------------------- - tableResult.getJobClient().ifPresent(JobClient::cancel); - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } @Test @@ -952,8 +1172,140 @@ public void testPerPartitionWatermarkWithIdleSource() throws Exception { // ------------- cleanup ------------------- - tableResult.getJobClient().ifPresent(JobClient::cancel); - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); + } + + @Test + public void testLatestOffsetStrategyResume() throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "latest_offset_resume_topic_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 6, 1); + env.setParallelism(1); + + // ---------- Produce data into Kafka's partition 0-6 ------------------- + + String groupId = getStandardProps().getProperty("group.id"); + String bootstraps = getBootstrapServers(); + + final String createTable = + String.format( + "CREATE TABLE kafka (\n" + + " `partition_id` INT,\n" + + " `value` INT\n" + + ") WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'properties.group.id' = '%s',\n" + + " 'scan.startup.mode' = 'latest-offset',\n" + + " 'sink.partitioner' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + topic, bootstraps, groupId, TestPartitioner.class.getName(), format); + + tEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO kafka VALUES (0, 0), (1, 0), (2, 0), (3, 0), (4, 0), (5, 0)"; + tEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Kafka ------------------- + + String createSink = + "CREATE TABLE MySink(\n" + + " `id` INT,\n" + + " `value` INT\n" + + ") WITH (\n" + + " 'connector' = 'values'\n" + + ")"; + tEnv.executeSql(createSink); + + String executeInsert = "INSERT INTO MySink SELECT `partition_id`, `value` FROM kafka"; + TableResult tableResult = tEnv.executeSql(executeInsert); + + // ---------- Produce data into Kafka's partition 0-2 ------------------- + + String moreValues = "INSERT INTO kafka VALUES (0, 1), (1, 1), (2, 1)"; + tEnv.executeSql(moreValues).await(); + + final List expected = Arrays.asList("+I[0, 1]", "+I[1, 1]", "+I[2, 1]"); + KafkaTableTestUtils.waitingExpectedResults("MySink", expected, Duration.ofSeconds(5)); + + // ---------- Stop the consume job with savepoint ------------------- + + String savepointBasePath = getTempDirPath(topic + "-savepoint"); + assert tableResult.getJobClient().isPresent(); + JobClient client = tableResult.getJobClient().get(); + String savepointPath = + client.stopWithSavepoint(false, savepointBasePath, SavepointFormatType.DEFAULT) + .get(); + + // ---------- Produce data into Kafka's partition 0-5 ------------------- + + String produceValuesBeforeResume = + "INSERT INTO kafka VALUES (0, 2), (1, 2), (2, 2), (3, 1), (4, 1), (5, 1)"; + tEnv.executeSql(produceValuesBeforeResume).await(); + + // ---------- Resume the consume job from savepoint ------------------- + + Configuration configuration = new Configuration(); + configuration.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + + tEnv.executeSql(createTable); + tEnv.executeSql(createSink); + tableResult = tEnv.executeSql(executeInsert); + + final List afterResumeExpected = + Arrays.asList( + "+I[0, 1]", + "+I[1, 1]", + "+I[2, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 2]", + "+I[3, 1]", + "+I[4, 1]", + "+I[5, 1]"); + KafkaTableTestUtils.waitingExpectedResults( + "MySink", afterResumeExpected, Duration.ofSeconds(5)); + + // ---------- Produce data into Kafka's partition 0-5 ------------------- + + String produceValuesAfterResume = + "INSERT INTO kafka VALUES (0, 3), (1, 3), (2, 3), (3, 2), (4, 2), (5, 2)"; + this.tEnv.executeSql(produceValuesAfterResume).await(); + + final List afterProduceExpected = + Arrays.asList( + "+I[0, 1]", + "+I[1, 1]", + "+I[2, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 2]", + "+I[3, 1]", + "+I[4, 1]", + "+I[5, 1]", + "+I[0, 3]", + "+I[1, 3]", + "+I[2, 3]", + "+I[3, 2]", + "+I[4, 2]", + "+I[5, 2]"); + KafkaTableTestUtils.waitingExpectedResults( + "MySink", afterProduceExpected, Duration.ofSeconds(5)); + + // ------------- cleanup ------------------- + + cancelJob(tableResult); + cleanupTopic(topic); } @Test @@ -969,7 +1321,7 @@ public void testStartFromGroupOffsetsEarliest() throws Exception { @Test public void testStartFromGroupOffsetsNone() { Assertions.assertThatThrownBy(() -> testStartFromGroupOffsetsWithNoneResetStrategy()) - .satisfies(FlinkAssertions.anyCauseMatches(NoOffsetForPartitionException.class)); + .satisfies(anyCauseMatches(NoOffsetForPartitionException.class)); } private List appendNewData( @@ -1085,10 +1437,8 @@ private void testStartFromGroupOffsets(String resetStrategy) throws Exception { KafkaTableTestUtils.waitingExpectedResults(sinkName, expected, Duration.ofSeconds(15)); } finally { // ------------- cleanup ------------------- - if (tableResult != null) { - tableResult.getJobClient().ifPresent(JobClient::cancel); - } - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } } @@ -1107,10 +1457,8 @@ private void testStartFromGroupOffsetsWithNoneResetStrategy() tableResult.await(); } finally { // ------------- cleanup ------------------- - if (tableResult != null) { - tableResult.getJobClient().ifPresent(JobClient::cancel); - } - deleteTestTopic(topic); + cancelJob(tableResult); + cleanupTopic(topic); } } @@ -1166,4 +1514,33 @@ private static boolean isCausedByJobFinished(Throwable e) { return false; } } + + private void cleanupTopic(String topic) { + ignoreExceptions( + () -> deleteTestTopic(topic), + anyCauseMatches(UnknownTopicOrPartitionException.class)); + } + + private static void cancelJob(TableResult tableResult) { + if (tableResult != null && tableResult.getJobClient().isPresent()) { + ignoreExceptions( + () -> tableResult.getJobClient().get().cancel().get(), + anyCauseMatches(FlinkJobTerminatedWithoutCancellationException.class), + anyCauseMatches( + "MiniCluster is not yet running or has already been shut down.")); + } + } + + @SafeVarargs + private static void ignoreExceptions( + RunnableWithException runnable, ThrowingConsumer... ignoreIf) { + try { + runnable.run(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception ex) { + // check if the exception is one of the ignored ones + assertThat(ex).satisfiesAnyOf(ignoreIf); + } + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index f80a54fc9..cc7c73271 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -19,15 +19,16 @@ package org.apache.flink.streaming.connectors.kafka.table; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.util.DockerImageVersions; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.RecordsToDelete; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.admin.TopicListing; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -41,8 +42,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerImageName; import java.time.Duration; import java.util.ArrayList; @@ -65,15 +64,8 @@ public abstract class KafkaTableTestBase extends AbstractTestBase { @ClassRule public static final KafkaContainer KAFKA_CONTAINER = - new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) { - @Override - protected void doStart() { - super.doStart(); - if (LOG.isInfoEnabled()) { - this.followOutput(new Slf4jLogConsumer(LOG)); - } - } - }.withEmbeddedZookeeper() + KafkaUtil.createKafkaContainer(KafkaTableTestBase.class) + .withEmbeddedZookeeper() .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) .withEnv( "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", @@ -170,6 +162,24 @@ public void deleteTestTopic(String topic) { } } + public void deleteRecords(String topic, Map partitionOffsetsToDelete) { + Map properties = new HashMap<>(); + properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, getBootstrapServers()); + + try (AdminClient admin = AdminClient.create(properties)) { + Map recordsToDelete = new HashMap<>(); + for (Map.Entry entry : partitionOffsetsToDelete.entrySet()) { + TopicPartition partition = new TopicPartition(topic, entry.getKey()); + RecordsToDelete records = RecordsToDelete.beforeOffset(entry.getValue()); + recordsToDelete.put(partition, records); + } + admin.deleteRecords(recordsToDelete).all().get(); + } catch (Exception e) { + throw new IllegalStateException( + String.format("Fail to delete records on topic [%s].", topic), e); + } + } + // ------------------------ For Debug Logging Purpose ---------------------------------- private void scheduleTimeoutLogger(Duration period, Runnable loggingAction) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java index c1aada46d..e4a5ba628 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestUtils.java @@ -41,6 +41,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.HamcrestCondition.matching; @@ -68,6 +69,23 @@ public static List collectRows(Table table, int expectedSize) throws Except return collectedRows; } + /** + * Variant of {@link #collectRows(Table, int)} for bounded queries. This should not run + * indefinitely if there is a bounded number of returned rows. + */ + public static List collectAllRows(Table table) throws Exception { + final TableResult result = table.execute(); + + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + + return collectedRows; + } + public static List readLines(String resource) throws IOException { final URL url = KafkaChangelogTableITCase.class.getClassLoader().getResource(resource); assertThat(url).isNotNull(); @@ -81,8 +99,11 @@ public static void waitingExpectedResults( Collections.sort(expected); CommonTestUtils.waitUtil( () -> { - List actual = TestValuesTableFactory.getResults(sinkName); - Collections.sort(actual); + List actual = + TestValuesTableFactory.getResults(sinkName).stream() + .map(KafkaTableTestUtils::rowToString) + .sorted() + .collect(Collectors.toList()); return expected.equals(actual); }, timeout, @@ -107,4 +128,12 @@ public static void comparedWithKeyAndOrder( matching(TableTestMatchers.deepEqualTo(expectedData.get(key), false))); } } + + private static String rowToString(Object o) { + if (o instanceof Row) { + return ((Row) o).toString(); + } else { + return o.toString(); + } + } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index b6f2788e9..0640b9ad7 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -20,7 +20,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.sink2.StatefulSink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -40,6 +41,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -150,7 +152,7 @@ public ReducingUpsertWriterTest(boolean enableObjectReuse) { @Test public void testWriteData() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); - final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); // write 4 records which doesn't trigger batch size writeData(bufferedWriter, new ReusableIterator(0, 4)); @@ -217,11 +219,12 @@ public void testWriteData() throws Exception { @Test public void testFlushDataWhenCheckpointing() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); - final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); // write all data, there should be 3 records are still buffered writeData(bufferedWriter, new ReusableIterator(0, 4)); // snapshot should flush the buffer bufferedWriter.flush(true); + assertThat(writer.flushed).isTrue(); HashMap> expected = new HashMap<>(); expected.put( @@ -261,6 +264,50 @@ public void testFlushDataWhenCheckpointing() throws Exception { compareCompactedResult(expected, writer.rowDataCollectors); } + @Test + public void testWriteDataWithNullTimestamp() throws Exception { + final MockedSinkWriter writer = new MockedSinkWriter(); + final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); + + bufferedWriter.write( + GenericRowData.ofKind( + INSERT, + 1001, + StringData.fromString("Java public for dummies"), + StringData.fromString("Tan Ah Teck"), + 11.11, + 11, + null), + new SinkWriter.Context() { + @Override + public long currentWatermark() { + throw new UnsupportedOperationException("Not implemented."); + } + + @Override + public Long timestamp() { + return null; + } + }); + + bufferedWriter.flush(true); + + final HashMap> expected = new HashMap<>(); + expected.put( + 1001, + Collections.singletonList( + GenericRowData.ofKind( + UPDATE_AFTER, + 1001, + StringData.fromString("Java public for dummies"), + StringData.fromString("Tan Ah Teck"), + 11.11, + 11, + null))); + + compareCompactedResult(expected, writer.rowDataCollectors); + } + private void compareCompactedResult( Map> expected, List actual) { Map> actualMap = new HashMap<>(); @@ -276,7 +323,7 @@ private void compareCompactedResult( } } - private void writeData(ReducingUpsertWriter writer, Iterator iterator) + private void writeData(ReducingUpsertWriter writer, Iterator iterator) throws Exception { while (iterator.hasNext()) { RowData next = iterator.next(); @@ -298,7 +345,7 @@ public Long timestamp() { } @SuppressWarnings("unchecked") - private ReducingUpsertWriter createBufferedWriter(MockedSinkWriter sinkWriter) { + private ReducingUpsertWriter createBufferedWriter(MockedSinkWriter sinkWriter) { TypeInformation typeInformation = (TypeInformation) new SinkRuntimeProviderContext(false) @@ -326,7 +373,10 @@ public ScheduledFuture registerTimer( } private static class MockedSinkWriter - implements StatefulSink.StatefulSinkWriter { + implements TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter< + RowData, Void, Void> { + + boolean flushed = false; transient List rowDataCollectors; @@ -334,16 +384,21 @@ private static class MockedSinkWriter rowDataCollectors = new ArrayList<>(); } - @Override public void write(RowData element, Context context) throws IOException, InterruptedException { - assertThat(Instant.ofEpochMilli(context.timestamp())) - .isEqualTo(element.getTimestamp(TIMESTAMP_INDICES, 3).toInstant()); + // Allow comparison between null timestamps + if (context.timestamp() == null) { + assertThat(element.getTimestamp(TIMESTAMP_INDICES, 3)).isNull(); + } else { + assertThat(Instant.ofEpochMilli(context.timestamp())) + .isEqualTo(element.getTimestamp(TIMESTAMP_INDICES, 3).toInstant()); + } rowDataCollectors.add(element); } - @Override - public void flush(boolean endOfInput) throws IOException, InterruptedException {} + public void flush(boolean endOfInput) throws IOException, InterruptedException { + flushed = true; + } @Override public void close() throws Exception {} @@ -352,6 +407,11 @@ public void close() throws Exception {} public List snapshotState(long checkpointId) throws IOException { return null; } + + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return null; + } } private class ReusableIterator implements Iterator { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 5caaaa0ac..abde43dca 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -21,11 +21,14 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.KafkaSink; import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.KafkaSourceTestUtils; import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.RowDataToAvroConverters; @@ -36,6 +39,7 @@ import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.testutils.MockPartitionOffsetsRetriever; import org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperatorFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; @@ -64,23 +68,32 @@ import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.TestLogger; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import javax.annotation.Nullable; + import java.util.Arrays; 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.function.Consumer; +import java.util.regex.Pattern; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; +import static org.apache.flink.table.factories.FactoryUtil.SOURCE_PARALLELISM; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link UpsertKafkaDynamicTableFactory}. */ public class UpsertKafkaDynamicTableFactoryTest extends TestLogger { @@ -157,8 +170,73 @@ public void testTableSource() { SOURCE_KEY_FIELDS, SOURCE_VALUE_FIELDS, null, - SOURCE_TOPIC, - UPSERT_KAFKA_SOURCE_PROPERTIES); + Collections.singletonList(SOURCE_TOPIC), + UPSERT_KAFKA_SOURCE_PROPERTIES, + null); + assertThat(actualSource).isEqualTo(expectedSource); + + final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; + ScanTableSource.ScanRuntimeProvider provider = + actualUpsertKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertKafkaSource(provider); + } + + @Test + public void testTableSourceWithParallelism() { + final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType(); + // Construct table source using options and table source factory + final Map modifiedOptions = + getModifiedOptions( + getFullSourceOptions(), + options -> options.put(SOURCE_PARALLELISM.key(), "100")); + final DynamicTableSource actualSource = createTableSource(SOURCE_SCHEMA, modifiedOptions); + + final KafkaDynamicSource expectedSource = + createExpectedScanSource( + producedDataType, + keyDecodingFormat, + valueDecodingFormat, + SOURCE_KEY_FIELDS, + SOURCE_VALUE_FIELDS, + null, + Collections.singletonList(SOURCE_TOPIC), + UPSERT_KAFKA_SOURCE_PROPERTIES, + 100); + assertThat(actualSource).isEqualTo(expectedSource); + + final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; + ScanTableSource.ScanRuntimeProvider provider = + actualUpsertKafkaSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final DataStreamScanProvider sourceProvider = (DataStreamScanProvider) provider; + assertThat(sourceProvider.getParallelism()).isPresent(); + assertThat(sourceProvider.getParallelism()).hasValue(100); + } + + @Test + public void testTableSourceWithTopicList() { + final Map modifiedOptions = + getModifiedOptions( + getFullSourceOptions(), + options -> { + options.put( + "topic", String.format("%s;%s", SOURCE_TOPIC, SOURCE_TOPIC)); + }); + final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType(); + // Construct table source using options and table source factory + final DynamicTableSource actualSource = createTableSource(SOURCE_SCHEMA, modifiedOptions); + + final KafkaDynamicSource expectedSource = + createExpectedScanSource( + producedDataType, + keyDecodingFormat, + valueDecodingFormat, + SOURCE_KEY_FIELDS, + SOURCE_VALUE_FIELDS, + null, + Arrays.asList(SOURCE_TOPIC, SOURCE_TOPIC), + UPSERT_KAFKA_SOURCE_PROPERTIES, + null); assertThat(actualSource).isEqualTo(expectedSource); final KafkaDynamicSource actualUpsertKafkaSource = (KafkaDynamicSource) actualSource; @@ -170,7 +248,14 @@ public void testTableSource() { @Test public void testTableSink() { // Construct table sink using options and table sink factory. - final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, getFullSinkOptions()); + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + }); + final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); final DynamicTableSink expectedSink = createExpectedSink( @@ -180,11 +265,55 @@ public void testTableSink() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, - null); + null, + "kafka-sink"); + + // Test sink format. + final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; + assertThat(actualSink).isEqualTo(expectedSink); + + // Test kafka producer. + DynamicTableSink.SinkRuntimeProvider provider = + actualUpsertKafkaSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkFunctionProvider = (SinkV2Provider) provider; + final Sink sink = sinkFunctionProvider.createSink(); + assertThat(sink).isInstanceOf(KafkaSink.class); + } + + @Test + public void testTableSinkWithTopicList() { + // Construct table sink using options and table sink factory. + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + options.put("topic", String.format("%s;%s", SINK_TOPIC, SINK_TOPIC)); + }); + final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); + + final DynamicTableSink expectedSink = + createExpectedSink( + SINK_SCHEMA.toPhysicalRowDataType(), + keyEncodingFormat, + valueEncodingFormat, + SINK_KEY_FIELDS, + SINK_VALUE_FIELDS, + null, + Arrays.asList(SINK_TOPIC, SINK_TOPIC), + null, + UPSERT_KAFKA_SINK_PROPERTIES, + DeliveryGuarantee.EXACTLY_ONCE, + SinkBufferFlushMode.DISABLED, + null, + "kafka-sink"); // Test sink format. final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; @@ -211,6 +340,8 @@ public void testBufferedTableSink() { options -> { options.put("sink.buffer-flush.max-rows", "100"); options.put("sink.buffer-flush.interval", "1s"); + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); })); final DynamicTableSink expectedSink = @@ -221,11 +352,13 @@ public void testBufferedTableSink() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, new SinkBufferFlushMode(100, 1000L), - null); + null, + "kafka-sink"); // Test sink format. final KafkaDynamicSink actualUpsertKafkaSink = (KafkaDynamicSink) actualSink; @@ -258,7 +391,12 @@ public void testBufferedTableSink() { public void testTableSinkWithParallelism() { final Map modifiedOptions = getModifiedOptions( - getFullSinkOptions(), options -> options.put("sink.parallelism", "100")); + getFullSinkOptions(), + options -> { + options.put("sink.parallelism", "100"); + options.put("sink.delivery-guarantee", "exactly-once"); + options.put("sink.transactional-id-prefix", "kafka-sink"); + }); final DynamicTableSink actualSink = createTableSink(SINK_SCHEMA, modifiedOptions); final DynamicTableSink expectedSink = @@ -269,11 +407,13 @@ public void testTableSinkWithParallelism() { SINK_KEY_FIELDS, SINK_VALUE_FIELDS, null, - SINK_TOPIC, + Collections.singletonList(SINK_TOPIC), + null, UPSERT_KAFKA_SINK_PROPERTIES, - DeliveryGuarantee.AT_LEAST_ONCE, + DeliveryGuarantee.EXACTLY_ONCE, SinkBufferFlushMode.DISABLED, - 100); + 100, + "kafka-sink"); assertThat(actualSink).isEqualTo(expectedSink); final DynamicTableSink.SinkRuntimeProvider provider = @@ -390,6 +530,138 @@ private SerializationSchema createConfluentAvroSerSchema( RowDataToAvroConverters.createConverter(rowType)); } + // -------------------------------------------------------------------------------------------- + // Bounded end-offset tests + // -------------------------------------------------------------------------------------------- + + @Test + public void testBoundedSpecificOffsetsValidate() { + final Map options = getFullSourceOptions(); + options.put( + KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), + ScanBoundedMode.SPECIFIC_OFFSETS.toString()); + + assertThatThrownBy(() -> createTableSource(SOURCE_SCHEMA, options)) + .isInstanceOf(ValidationException.class) + .cause() + .hasMessageContaining( + "'scan.bounded.specific-offsets' is required in 'specific-offsets' bounded mode but missing."); + } + + @Test + public void testBoundedSpecificOffsets() { + testBoundedOffsets( + ScanBoundedMode.SPECIFIC_OFFSETS, + options -> { + options.put("scan.bounded.specific-offsets", "partition:0,offset:2"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.noInteractions()); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, 2L); + }); + } + + @Test + public void testBoundedLatestOffset() { + testBoundedOffsets( + ScanBoundedMode.LATEST_OFFSET, + options -> {}, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + long endOffsets = 123L; + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.latest( + (tps) -> + Collections.singletonMap( + partition, endOffsets))); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, endOffsets); + }); + } + + @Test + public void testBoundedGroupOffsets() { + testBoundedOffsets( + ScanBoundedMode.GROUP_OFFSETS, + options -> { + options.put("properties.group.id", "dummy"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.noInteractions()); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, KafkaPartitionSplit.COMMITTED_OFFSET); + }); + } + + @Test + public void testBoundedTimestamp() { + testBoundedOffsets( + ScanBoundedMode.TIMESTAMP, + options -> { + options.put("scan.bounded.timestamp-millis", "1"); + }, + source -> { + assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); + OffsetsInitializer offsetsInitializer = + KafkaSourceTestUtils.getStoppingOffsetsInitializer(source); + TopicPartition partition = new TopicPartition(SOURCE_TOPIC, 0); + long offsetForTimestamp = 123L; + Map partitionOffsets = + offsetsInitializer.getPartitionOffsets( + Collections.singletonList(partition), + MockPartitionOffsetsRetriever.timestampAndEnd( + partitions -> { + assertThat(partitions) + .containsOnlyKeys(partition) + .containsEntry(partition, 1L); + Map result = + new HashMap<>(); + result.put( + partition, + new OffsetAndTimestamp( + offsetForTimestamp, 1L)); + return result; + }, + partitions -> { + Map result = new HashMap<>(); + result.put( + partition, + // the end offset is bigger than given by + // timestamp + // to make sure the one for timestamp is + // used + offsetForTimestamp + 1000L); + return result; + })); + assertThat(partitionOffsets) + .containsOnlyKeys(partition) + .containsEntry(partition, offsetForTimestamp); + }); + } + // -------------------------------------------------------------------------------------------- // Negative tests // -------------------------------------------------------------------------------------------- @@ -498,6 +770,26 @@ public void testInvalidSinkBufferFlush() { })); } + @Test + public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { + thrown.expect(ValidationException.class); + thrown.expect( + containsCause( + new ValidationException( + "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE."))); + + final Map modifiedOptions = + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.remove(KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX.key()); + options.put( + KafkaConnectorOptions.DELIVERY_GUARANTEE.key(), + DeliveryGuarantee.EXACTLY_ONCE.toString()); + }); + createTableSink(SINK_SCHEMA, modifiedOptions); + } + // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @@ -595,8 +887,9 @@ private KafkaDynamicSource createExpectedScanSource( int[] keyFields, int[] valueFields, String keyPrefix, - String topic, - Properties properties) { + List topic, + Properties properties, + @Nullable Integer parallelism) { return new KafkaDynamicSource( producedDataType, keyDecodingFormat, @@ -604,7 +897,7 @@ private KafkaDynamicSource createExpectedScanSource( keyFields, valueFields, keyPrefix, - Collections.singletonList(topic), + topic, null, properties, StartupMode.EARLIEST, @@ -614,7 +907,8 @@ private KafkaDynamicSource createExpectedScanSource( Collections.emptyMap(), 0, true, - FactoryMocks.IDENTIFIER.asSummaryString()); + FactoryMocks.IDENTIFIER.asSummaryString(), + parallelism); } private static KafkaDynamicSink createExpectedSink( @@ -624,11 +918,13 @@ private static KafkaDynamicSink createExpectedSink( int[] keyProjection, int[] valueProjection, String keyPrefix, - String topic, + List topics, + Pattern topicPattern, Properties properties, DeliveryGuarantee deliveryGuarantee, SinkBufferFlushMode flushMode, - Integer parallelism) { + Integer parallelism, + String transactionalIdPrefix) { return new KafkaDynamicSink( consumedDataType, consumedDataType, @@ -637,17 +933,18 @@ private static KafkaDynamicSink createExpectedSink( keyProjection, valueProjection, keyPrefix, - topic, + topics, + topicPattern, properties, null, deliveryGuarantee, true, flushMode, parallelism, - null); + transactionalIdPrefix); } - private void assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { + private KafkaSource assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { assertThat(provider).isInstanceOf(DataStreamScanProvider.class); final DataStreamScanProvider dataStreamScanProvider = (DataStreamScanProvider) provider; final Transformation transformation = @@ -662,5 +959,24 @@ private void assertKafkaSource(ScanTableSource.ScanRuntimeProvider provider) { (SourceTransformation) transformation; assertThat(sourceTransformation.getSource()).isInstanceOf(KafkaSource.class); + return (KafkaSource) sourceTransformation.getSource(); + } + + private void testBoundedOffsets( + ScanBoundedMode boundedMode, + Consumer> optionsConfig, + Consumer> validator) { + final Map options = getFullSourceOptions(); + options.put(KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), boundedMode.toString()); + optionsConfig.accept(options); + + final DynamicTableSource tableSource = createTableSource(SOURCE_SCHEMA, options); + assertThat(tableSource).isInstanceOf(KafkaDynamicSource.class); + ScanTableSource.ScanRuntimeProvider provider = + ((KafkaDynamicSource) tableSource) + .getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertThat(provider).isInstanceOf(DataStreamScanProvider.class); + final KafkaSource kafkaSource = assertKafkaSource(provider); + validator.accept(kafkaSource); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java index 109f4402a..1a6bf7e13 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java @@ -33,17 +33,20 @@ import java.time.Duration; import java.time.LocalDateTime; +import java.time.ZoneId; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; import static org.apache.flink.api.common.typeinfo.Types.INT; import static org.apache.flink.api.common.typeinfo.Types.LOCAL_DATE_TIME; import static org.apache.flink.api.common.typeinfo.Types.ROW_NAMED; import static org.apache.flink.api.common.typeinfo.Types.STRING; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectAllRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.collectRows; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.comparedWithKeyAndOrder; import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.waitingExpectedResults; @@ -195,6 +198,71 @@ public void testBufferedUpsertSink() throws Exception { deleteTestTopic(topic); } + @Test + public void testBufferedUpsertSinkWithoutAssigningWatermark() throws Exception { + final String topic = "buffered_upsert_topic_without_assigning_watermark_" + format; + createTestTopic(topic, 1, 1); + String bootstraps = getBootstrapServers(); + env.setParallelism(1); + + Table table = + tEnv.fromDataStream( + env.fromElements( + Row.of(1, null, "payload 1"), + Row.of(2, null, "payload 2"), + Row.of(3, null, "payload 3"), + Row.of(3, null, "payload")) + .returns( + ROW_NAMED( + new String[] {"k_id", "ts", "payload"}, + INT, + LOCAL_DATE_TIME, + STRING)), + Schema.newBuilder() + .column("k_id", DataTypes.INT()) + .column("ts", DataTypes.TIMESTAMP(3)) + .column("payload", DataTypes.STRING()) + .build()); + + final String createTable = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `k_id` INTEGER,\n" + + " `ts` TIMESTAMP(3),\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (k_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'sink.buffer-flush.max-rows' = '2',\n" + + " 'sink.buffer-flush.interval' = '100000',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'key.csv.null-literal' = '',\n" + + " 'value.csv.null-literal' = ''\n" + + ")", + topic, bootstraps, "csv", "csv"); + + tEnv.executeSql(createTable); + + table.executeInsert("upsert_kafka").await(); + + final List result = collectRows(tEnv.sqlQuery("SELECT * FROM upsert_kafka"), 3); + final List expected = + Arrays.asList( + changelogRow("+I", 1, null, "payload 1"), + changelogRow("+I", 2, null, "payload 2"), + changelogRow("+I", 3, null, "payload")); + + assertThat(result).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + @Test public void testSourceSinkWithKeyAndPartialValue() throws Exception { // we always use a different topic name for each parameterized topic, @@ -388,6 +456,202 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { deleteTestTopic(topic); } + @Test + public void testUpsertKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to offset=2 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'specific-offsets',\n" + + " 'scan.bounded.specific-offsets' = 'partition:0,offset:2'" + + ")", + topic, bootstraps, format, format); + tEnv.executeSql(createTableSql); + + // insert multiple values to have more records past offset=2 + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, 100, 'payload 1'),\n" + + " (1, 100, 'payload 1-new'),\n" + + " (2, 101, 'payload 2'),\n" + + " (3, 102, 'payload 3')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should only have records up to offset=2 + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + final List expected = + Arrays.asList( + changelogRow("+I", 1L, 100L, "payload 1"), + changelogRow("-U", 1L, 100L, "payload 1"), + changelogRow("+U", 1L, 100L, "payload 1-new")); + assertThat(results).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + + @Test + public void testUpsertKafkaSourceSinkWithBoundedTimestamp() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to timestamp 2023-03-10T14:00:00.000 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `timestamp` TIMESTAMP(3) METADATA,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'timestamp',\n" + + " 'scan.bounded.timestamp-millis' = '%d'" + + ")", + topic, + bootstraps, + format, + format, + LocalDateTime.parse("2023-03-10T14:00:00.000") + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + tEnv.executeSql(createTableSql); + + // insert multiple values with timestamp starting from 2023-03-08 up to 2023-03-11 + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, TIMESTAMP '2023-03-08 08:10:10.666', 100, 'payload 1'),\n" + + " (2, TIMESTAMP '2023-03-09 13:12:11.123', 101, 'payload 2'),\n" + + " (1, TIMESTAMP '2023-03-10 12:09:50.321', 100, 'payload 1-new'),\n" + + " (2, TIMESTAMP '2023-03-11 17:15:13.457', 101, 'payload 2-new')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should only have records up to timestamp 2023-03-10T14:00:00.000 + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + final List expected = + Arrays.asList( + changelogRow( + "+I", + 1L, + LocalDateTime.parse("2023-03-08T08:10:10.666"), + 100L, + "payload 1"), + changelogRow( + "+I", + 2L, + LocalDateTime.parse("2023-03-09T13:12:11.123"), + 101L, + "payload 2"), + changelogRow( + "-U", + 1L, + LocalDateTime.parse("2023-03-08T08:10:10.666"), + 100L, + "payload 1"), + changelogRow( + "+U", + 1L, + LocalDateTime.parse("2023-03-10T12:09:50.321"), + 100L, + "payload 1-new")); + assertThat(results).satisfies(matching(deepEqualTo(expected, true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + + /** + * Tests that setting bounded end offset that is before the earliest offset results in 0 + * results. + */ + @Test + public void testUpsertKafkaSourceSinkWithZeroLengthBoundedness() throws Exception { + final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); + createTestTopic(topic, 1, 1); + + // ---------- Produce an event time stream into Kafka ------------------- + final String bootstraps = getBootstrapServers(); + + // table with upsert-kafka connector, bounded mode up to timestamp 2023-03-10T14:00:00.000 + final String createTableSql = + String.format( + "CREATE TABLE upsert_kafka (\n" + + " `user_id` BIGINT,\n" + + " `timestamp` TIMESTAMP(3) METADATA,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING,\n" + + " PRIMARY KEY (event_id, user_id) NOT ENFORCED" + + ") WITH (\n" + + " 'connector' = 'upsert-kafka',\n" + + " 'topic' = '%s',\n" + + " 'properties.bootstrap.servers' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'value.format' = '%s',\n" + + " 'value.fields-include' = 'ALL',\n" + + " 'scan.bounded.mode' = 'timestamp',\n" + + " 'scan.bounded.timestamp-millis' = '%d'" + + ")", + topic, + bootstraps, + format, + format, + LocalDateTime.parse("2023-03-10T14:00:00.000") + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + tEnv.executeSql(createTableSql); + + // insert multiple values with timestamp starting from 2023-03-11 (which is past the bounded + // end timestamp) + final String insertValuesSql = + "INSERT INTO upsert_kafka\n" + + "VALUES\n" + + " (1, TIMESTAMP '2023-03-11 08:10:10.666', 100, 'payload 1'),\n" + + " (2, TIMESTAMP '2023-03-12 13:12:11.123', 101, 'payload 2'),\n" + + " (1, TIMESTAMP '2023-03-13 12:09:50.321', 100, 'payload 1-new'),\n" + + " (2, TIMESTAMP '2023-03-14 17:15:13.457', 101, 'payload 2-new')"; + tEnv.executeSql(insertValuesSql).await(); + + // results should be empty + final List results = collectAllRows(tEnv.sqlQuery("SELECT * from upsert_kafka")); + assertThat(results).satisfies(matching(deepEqualTo(Collections.emptyList(), true))); + + // ------------- cleanup ------------------- + + deleteTestTopic(topic); + } + private void wordCountToUpsertKafka(String wordCountTable) throws Exception { String bootstraps = getBootstrapServers(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index 83ee3fb1c..92978a783 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -18,26 +18,20 @@ package org.apache.flink.streaming.connectors.kafka.testutils; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironment; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import java.util.Collections; -import java.util.List; import java.util.Properties; import java.util.Random; @@ -103,8 +97,7 @@ public void cancel() { Properties props = new Properties(); props.putAll( - FlinkKafkaProducerBase.getPropertiesFromBrokerList( - testServer.getBrokerConnectionString())); + KafkaUtils.getPropertiesFromBrokerList(testServer.getBrokerConnectionString())); Properties secureProps = testServer.getSecureProperties(); if (secureProps != null) { props.putAll(testServer.getSecureProperties()); @@ -158,21 +151,19 @@ public InfiniteStringsGenerator(KafkaTestEnvironment server, String topic) { @Override public void run() { // we manually feed data into the Kafka sink - OneInputStreamOperatorTestHarness testHarness = null; - try { - Properties producerProperties = - FlinkKafkaProducerBase.getPropertiesFromBrokerList( - server.getBrokerConnectionString()); - producerProperties.setProperty("retries", "3"); - - StreamSink sink = - server.getProducerSink( - topic, - new SimpleStringSchema(), - producerProperties, - new FlinkFixedPartitioner<>()); - - testHarness = new OneInputStreamOperatorTestHarness<>(sink); + + Properties producerProperties = + KafkaUtils.getPropertiesFromBrokerList(server.getBrokerConnectionString()); + producerProperties.setProperty("retries", "3"); + + StreamSink sink = + server.getProducerSink( + topic, + new SimpleStringSchema(), + producerProperties, + new FlinkFixedPartitioner<>()); + try (OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(sink)) { testHarness.open(); final StringBuilder bld = new StringBuilder(); @@ -191,48 +182,15 @@ public void run() { } } catch (Throwable t) { this.error = t; - } finally { - if (testHarness != null) { - try { - testHarness.close(); - } catch (Throwable t) { - // ignore - } - } } } public void shutdown() { this.running = false; - this.interrupt(); } public Throwable getError() { return this.error; } - - private static class MockTransformation extends Transformation { - public MockTransformation() { - super("MockTransform", BasicTypeInfo.STRING_TYPE_INFO, 1); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } - - private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - return null; - } - } } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java new file mode 100644 index 000000000..eeb8ce047 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/KafkaUtils.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.util.NetUtils; + +import org.apache.kafka.clients.producer.ProducerConfig; + +import java.util.Properties; + +/** Some utilities methods around Kafka. */ +public class KafkaUtils { + public static Properties getPropertiesFromBrokerList(String brokerList) { + String[] elements = brokerList.split(","); + + // validate the broker addresses + for (String broker : elements) { + NetUtils.getCorrectHostnamePort(broker); + } + + Properties props = new Properties(); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList); + return props; + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java new file mode 100644 index 000000000..9947bc5b6 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockPartitionOffsetsRetriever.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; + +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; + +import java.util.Collection; +import java.util.Map; +import java.util.function.Function; + +/** Fake {@link OffsetsInitializer.PartitionOffsetsRetriever} for unit tests. */ +public final class MockPartitionOffsetsRetriever + implements OffsetsInitializer.PartitionOffsetsRetriever { + + /** Fake offsets retriever for a given set of topic partitions. */ + public interface OffsetsRetriever + extends Function, Map> {} + + /** + * Fake offsets retrieve for a given set of topic partitions and their target timestamp + * position. + */ + public interface TimestampOffsetsRetriever + extends Function, Map> {} + + public static final OffsetsRetriever UNSUPPORTED_RETRIEVAL = + partitions -> { + throw new UnsupportedOperationException("The method was not supposed to be called"); + }; + private final OffsetsRetriever committedOffsets; + private final OffsetsRetriever endOffsets; + private final OffsetsRetriever beginningOffsets; + private final TimestampOffsetsRetriever offsetsForTimes; + + public static MockPartitionOffsetsRetriever noInteractions() { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + UNSUPPORTED_RETRIEVAL, + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }); + } + + public static MockPartitionOffsetsRetriever timestampAndEnd( + TimestampOffsetsRetriever retriever, OffsetsRetriever endOffsets) { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, endOffsets, UNSUPPORTED_RETRIEVAL, retriever); + } + + public static MockPartitionOffsetsRetriever latest(OffsetsRetriever endOffsets) { + return new MockPartitionOffsetsRetriever( + UNSUPPORTED_RETRIEVAL, + endOffsets, + UNSUPPORTED_RETRIEVAL, + partitions -> { + throw new UnsupportedOperationException( + "The method was not supposed to be called"); + }); + } + + private MockPartitionOffsetsRetriever( + OffsetsRetriever committedOffsets, + OffsetsRetriever endOffsets, + OffsetsRetriever beginningOffsets, + TimestampOffsetsRetriever offsetsForTimes) { + this.committedOffsets = committedOffsets; + this.endOffsets = endOffsets; + this.beginningOffsets = beginningOffsets; + this.offsetsForTimes = offsetsForTimes; + } + + @Override + public Map committedOffsets(Collection partitions) { + return committedOffsets.apply(partitions); + } + + @Override + public Map endOffsets(Collection partitions) { + return endOffsets.apply(partitions); + } + + @Override + public Map beginningOffsets(Collection partitions) { + return beginningOffsets.apply(partitions); + } + + @Override + public Map offsetsForTimes( + Map timestampsToSearch) { + return offsetsForTimes.apply(timestampsToSearch); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java new file mode 100644 index 000000000..cdaec31c1 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThreadContextClassLoader.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import java.io.Closeable; + +/** + * Utility class to temporarily use a different classloader as the thread context classloader. + * + *

      Temporarily copied from flink-core to avoid dependency on flink-core. + */ +public class ThreadContextClassLoader implements Closeable { + + private final ClassLoader originalThreadContextClassLoader; + + public ThreadContextClassLoader(ClassLoader newThreadContextClassLoader) { + this.originalThreadContextClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(newThreadContextClassLoader); + } + + @Override + public void close() { + Thread.currentThread().setContextClassLoader(originalThreadContextClassLoader); + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java new file mode 100644 index 000000000..0815e2990 --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerMatchers.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; + +import org.hamcrest.CoreMatchers; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.hamcrest.TypeSafeMatcher; + +import java.util.function.Predicate; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A Collection of useful {@link Matcher}s for {@link TypeSerializer} and {@link + * TypeSerializerSchemaCompatibility}. + * + *

      Note copied from Flink 1.19. Remove when we drop 1.19 support. + */ +public final class TypeSerializerMatchers { + + private TypeSerializerMatchers() {} + + // ------------------------------------------------------------------------------------------------------------- + // Matcher Factories + // ------------------------------------------------------------------------------------------------------------- + + /** + * Matches {@code compatibleAsIs} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code compatibleAsIs} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isCompatibleAsIs() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isCompatibleAsIs, + "type serializer schema that is a compatible as is"); + } + + /** + * Matches {@code isIncompatible} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isIncompatible} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isIncompatible() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isIncompatible, + "type serializer schema that is incompatible"); + } + + /** + * Matches {@code isCompatibleAfterMigration} {@link TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleAfterMigration} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static Matcher> isCompatibleAfterMigration() { + return propertyMatcher( + TypeSerializerSchemaCompatibility::isCompatibleAfterMigration, + "type serializer schema that is compatible after migration"); + } + + /** + * Matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + * + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static + Matcher> isCompatibleWithReconfiguredSerializer() { + @SuppressWarnings("unchecked") + Matcher> anything = + (Matcher>) (Matcher) CoreMatchers.anything(); + + return new CompatibleAfterReconfiguration<>(anything); + } + + /** + * Matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + * + * @param reconfiguredSerializerMatcher matches the reconfigured serializer. + * @param element type + * @return a {@code Matcher} that matches {@code isCompatibleWithReconfiguredSerializer} {@link + * TypeSerializerSchemaCompatibility}. + */ + public static + Matcher> isCompatibleWithReconfiguredSerializer( + Matcher> reconfiguredSerializerMatcher) { + + return new CompatibleAfterReconfiguration<>(reconfiguredSerializerMatcher); + } + + /** + * Matches if the expected {@code TypeSerializerSchemaCompatibility} has the same compatibility + * as {@code expectedCompatibility}. + * + * @param expectedCompatibility the compatibility to match to. + * @param element type. + * @return a {@code Matcher} that matches if it has the same compatibility as {@code + * expectedCompatibility}. + */ + public static Matcher> hasSameCompatibilityAs( + TypeSerializerSchemaCompatibility expectedCompatibility) { + + return new SchemaCompatibilitySameAs<>(expectedCompatibility); + } + + // ------------------------------------------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------------------------------------------- + + private static Matcher propertyMatcher( + Predicate predicate, String matcherDescription) { + return new TypeSafeMatcher() { + + @Override + protected boolean matchesSafely(T item) { + return predicate.test(item); + } + + @Override + public void describeTo(Description description) { + description.appendText(matcherDescription); + } + }; + } + + // ------------------------------------------------------------------------------------------------------------- + // Matchers + // ------------------------------------------------------------------------------------------------------------- + + private static final class CompatibleAfterReconfiguration + extends TypeSafeDiagnosingMatcher> { + + private final Matcher> reconfiguredSerializerMatcher; + + private CompatibleAfterReconfiguration( + Matcher> reconfiguredSerializerMatcher) { + this.reconfiguredSerializerMatcher = checkNotNull(reconfiguredSerializerMatcher); + } + + @Override + protected boolean matchesSafely( + TypeSerializerSchemaCompatibility item, Description mismatchDescription) { + if (!item.isCompatibleWithReconfiguredSerializer()) { + mismatchDescription.appendText( + "serializer schema is not compatible with a reconfigured serializer"); + return false; + } + TypeSerializer reconfiguredSerializer = item.getReconfiguredSerializer(); + if (!reconfiguredSerializerMatcher.matches(reconfiguredSerializer)) { + reconfiguredSerializerMatcher.describeMismatch( + reconfiguredSerializer, mismatchDescription); + return false; + } + return true; + } + + @Override + public void describeTo(Description description) { + description + .appendText("type serializer schema that is compatible after reconfiguration,") + .appendText("with a reconfigured serializer matching ") + .appendDescriptionOf(reconfiguredSerializerMatcher); + } + } + + private static class SchemaCompatibilitySameAs + extends TypeSafeMatcher> { + + private final TypeSerializerSchemaCompatibility expectedCompatibility; + + private SchemaCompatibilitySameAs( + TypeSerializerSchemaCompatibility expectedCompatibility) { + this.expectedCompatibility = checkNotNull(expectedCompatibility); + } + + @Override + protected boolean matchesSafely( + TypeSerializerSchemaCompatibility testResultCompatibility) { + if (expectedCompatibility.isCompatibleAsIs()) { + return testResultCompatibility.isCompatibleAsIs(); + } else if (expectedCompatibility.isIncompatible()) { + return testResultCompatibility.isIncompatible(); + } else if (expectedCompatibility.isCompatibleAfterMigration()) { + return testResultCompatibility.isCompatibleAfterMigration(); + } else if (expectedCompatibility.isCompatibleWithReconfiguredSerializer()) { + return testResultCompatibility.isCompatibleWithReconfiguredSerializer(); + } + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText("same compatibility as ").appendValue(expectedCompatibility); + } + } +} diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java new file mode 100644 index 000000000..ab3baadef --- /dev/null +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/TypeSerializerUpgradeTestBase.java @@ -0,0 +1,584 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +package org.apache.flink.streaming.connectors.kafka.testutils; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.typeutils.ClassRelocator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; + +import org.assertj.core.api.AssertionsForClassTypes; +import org.assertj.core.api.HamcrestCondition; +import org.hamcrest.Matcher; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collection; +import java.util.Set; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; +import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.hamcrest.CoreMatchers.not; + +/** + * A test base for testing {@link TypeSerializer} upgrades. + * + *

      You can run {@link #generateTestSetupFiles(TestSpecification)} on a Flink branch to + * (re-)generate the test data files. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class TypeSerializerUpgradeTestBase { + + public static final FlinkVersion CURRENT_VERSION = FlinkVersion.current(); + + public static final Set MIGRATION_VERSIONS = + FlinkVersion.rangeOf(FlinkVersion.v1_11, CURRENT_VERSION); + + // ------------------------------------------------------------------------------ + // APIs + // ------------------------------------------------------------------------------ + + /** + * Creates a collection of {@link TestSpecification} which will be used as input for + * parametrized tests. + */ + public abstract Collection> createTestSpecifications() throws Exception; + + /** + * Setup code for a {@link TestSpecification}. This creates the serializer before upgrade and + * test data, that will be written by the created pre-upgrade {@link TypeSerializer}. + */ + public interface PreUpgradeSetup { + + /** Creates a pre-upgrade {@link TypeSerializer}. */ + TypeSerializer createPriorSerializer(); + + /** Creates test data that will be written using the pre-upgrade {@link TypeSerializer}. */ + PreviousElementT createTestData(); + } + + /** + * Verification code for a {@link TestSpecification}. This creates the "upgraded" {@link + * TypeSerializer} and provides matchers for comparing the deserialized test data and for the + * {@link TypeSerializerSchemaCompatibility}. + */ + public interface UpgradeVerifier { + + /** Creates a post-upgrade {@link TypeSerializer}. */ + TypeSerializer createUpgradedSerializer(); + + /** Returns a {@link Matcher} for asserting the deserialized test data. */ + Matcher testDataMatcher(); + + /** + * Returns a {@link Matcher} for comparing the {@link TypeSerializerSchemaCompatibility} + * that the serializer upgrade produced with an expected {@link + * TypeSerializerSchemaCompatibility}. + */ + Matcher> schemaCompatibilityMatcher( + FlinkVersion version); + } + + private static class ClassLoaderSafePreUpgradeSetup + implements PreUpgradeSetup { + + private final PreUpgradeSetup delegateSetup; + private final ClassLoader setupClassloader; + + ClassLoaderSafePreUpgradeSetup( + Class> delegateSetupClass) + throws Exception { + checkNotNull(delegateSetupClass); + Class> relocatedDelegateSetupClass = + ClassRelocator.relocate(delegateSetupClass); + + this.setupClassloader = relocatedDelegateSetupClass.getClassLoader(); + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + this.delegateSetup = relocatedDelegateSetupClass.newInstance(); + } + } + + @Override + public TypeSerializer createPriorSerializer() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + return delegateSetup.createPriorSerializer(); + } + } + + @Override + public PreviousElementT createTestData() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(setupClassloader)) { + return delegateSetup.createTestData(); + } + } + } + + private static class ClassLoaderSafeUpgradeVerifier + implements UpgradeVerifier { + + private final UpgradeVerifier delegateVerifier; + private final ClassLoader verifierClassloader; + + ClassLoaderSafeUpgradeVerifier( + Class> delegateVerifierClass) + throws Exception { + checkNotNull(delegateVerifierClass); + Class> relocatedDelegateVerifierClass = + ClassRelocator.relocate(delegateVerifierClass); + + this.verifierClassloader = relocatedDelegateVerifierClass.getClassLoader(); + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + this.delegateVerifier = relocatedDelegateVerifierClass.newInstance(); + } + } + + @Override + public TypeSerializer createUpgradedSerializer() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.createUpgradedSerializer(); + } + } + + @Override + public Matcher testDataMatcher() { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.testDataMatcher(); + } + } + + @Override + public Matcher> + schemaCompatibilityMatcher(FlinkVersion version) { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(verifierClassloader)) { + return delegateVerifier.schemaCompatibilityMatcher(version); + } + } + } + + /** + * Specification of one test scenario. This mainly needs a {@link PreUpgradeSetup} and {@link + * UpgradeVerifier}. + */ + public static class TestSpecification { + private final String name; + private final FlinkVersion flinkVersion; + private final ClassLoaderSafePreUpgradeSetup setup; + private final ClassLoaderSafeUpgradeVerifier verifier; + + public TestSpecification( + String name, + FlinkVersion flinkVersion, + Class> setupClass, + Class> verifierClass) + throws Exception { + this.name = checkNotNull(name); + this.flinkVersion = checkNotNull(flinkVersion); + this.setup = new ClassLoaderSafePreUpgradeSetup<>(setupClass); + this.verifier = new ClassLoaderSafeUpgradeVerifier<>(verifierClass); + } + + @Override + public String toString() { + return name + " / " + flinkVersion; + } + } + + // ------------------------------------------------------------------------------ + // Test file generation + // ------------------------------------------------------------------------------ + + private static final int INITIAL_OUTPUT_BUFFER_SIZE = 64; + + /** + * Execute this test to generate test files. Remember to be using the correct branch when + * generating the test files, e.g. to generate test files for {@link FlinkVersion#v1_8}, you + * should be under the release-1.8 branch. + */ + @Disabled + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void generateTestSetupFiles( + TestSpecification testSpecification) + throws Exception { + Files.createDirectories(getSerializerSnapshotFilePath(testSpecification).getParent()); + + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.setup.setupClassloader)) { + TypeSerializer priorSerializer = + testSpecification.setup.createPriorSerializer(); + + // first, use the serializer to write test data + // NOTE: it is important that we write test data first, because some serializers' + // configuration + // mutates only after being used for serialization (e.g. dynamic type + // registrations for Pojo / Kryo) + DataOutputSerializer testDataOut = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + priorSerializer.serialize(testSpecification.setup.createTestData(), testDataOut); + writeContentsTo( + getGenerateDataFilePath(testSpecification), testDataOut.getCopyOfBuffer()); + + // ... then write the serializer snapshot + DataOutputSerializer serializerSnapshotOut = + new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializerSnapshot(serializerSnapshotOut, priorSerializer, CURRENT_VERSION); + writeContentsTo( + getGenerateSerializerSnapshotFilePath(testSpecification), + serializerSnapshotOut.getCopyOfBuffer()); + } + } + + // ------------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------------ + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void restoreSerializerIsValid( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + assumeThat(TypeSerializerSchemaCompatibility.incompatible()) + .as( + "This test only applies for test specifications that verify an upgraded serializer that is not incompatible.") + .is( + HamcrestCondition.matching( + not( + testSpecification.verifier.schemaCompatibilityMatcher( + testSpecification.flinkVersion)))); + + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + + TypeSerializer restoredSerializer = + restoredSerializerSnapshot.restoreSerializer(); + assertSerializerIsValid( + restoredSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerHasExpectedSchemaCompatibility( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + + AssertionsForClassTypes.assertThat(upgradeCompatibility) + .is( + HamcrestCondition.matching( + testSpecification.verifier.schemaCompatibilityMatcher( + testSpecification.flinkVersion))); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidAfterMigration( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that requires migration to be compatible.") + .is( + HamcrestCondition.matching( + TypeSerializerMatchers.isCompatibleAfterMigration())); + + // migrate the previous data schema, + TypeSerializer restoreSerializer = + restoredSerializerSnapshot.restoreSerializer(); + DataInputView migratedData = + readAndThenWriteData( + dataUnderTest(testSpecification), + restoreSerializer, + upgradedSerializer, + testSpecification.verifier.testDataMatcher()); + + // .. and then assert that the upgraded serializer is valid with the migrated data + assertSerializerIsValid( + upgradedSerializer, migratedData, testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidAfterReconfiguration( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that requires reconfiguration to be compatible.") + .is( + HamcrestCondition.matching( + TypeSerializerMatchers + .isCompatibleWithReconfiguredSerializer())); + + TypeSerializer reconfiguredUpgradedSerializer = + upgradeCompatibility.getReconfiguredSerializer(); + assertSerializerIsValid( + reconfiguredUpgradedSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + @ParameterizedTest(name = "Test Specification = {0}") + @MethodSource("createTestSpecifications") + void upgradedSerializerIsValidWhenCompatibleAsIs( + TestSpecification testSpecification) + throws Exception { + try (ThreadContextClassLoader ignored = + new ThreadContextClassLoader(testSpecification.verifier.verifierClassloader)) { + TypeSerializerSnapshot restoredSerializerSnapshot = + snapshotUnderTest(testSpecification); + TypeSerializer upgradedSerializer = + testSpecification.verifier.createUpgradedSerializer(); + + TypeSerializerSchemaCompatibility upgradeCompatibility = + restoredSerializerSnapshot.resolveSchemaCompatibility(upgradedSerializer); + assumeThat(upgradeCompatibility) + .as( + "This test only applies for test specifications that verify an upgraded serializer that is compatible as is.") + .is(HamcrestCondition.matching(TypeSerializerMatchers.isCompatibleAsIs())); + + assertSerializerIsValid( + upgradedSerializer, + dataUnderTest(testSpecification), + testSpecification.verifier.testDataMatcher()); + } + } + + /** + * Asserts that a given {@link TypeSerializer} is valid, given a {@link DataInputView} of + * serialized data. + * + *

      A serializer is valid, iff: + * + *

        + *
      • 1. The serializer can read and then write again the given serialized data. + *
      • 2. The serializer can produce a serializer snapshot which can be written and then read + * back again. + *
      • 3. The serializer's produced snapshot is capable of creating a restore serializer. + *
      • 4. The restore serializer created from the serializer snapshot can read and then write + * again data written by step 1. Given that the serializer is not a restore serializer + * already. + *
      + */ + private static void assertSerializerIsValid( + TypeSerializer serializer, DataInputView dataInput, Matcher testDataMatcher) + throws Exception { + + DataInputView serializedData = + readAndThenWriteData(dataInput, serializer, serializer, testDataMatcher); + TypeSerializerSnapshot snapshot = writeAndThenReadSerializerSnapshot(serializer); + TypeSerializer restoreSerializer = snapshot.restoreSerializer(); + serializedData = + readAndThenWriteData( + serializedData, restoreSerializer, restoreSerializer, testDataMatcher); + + TypeSerializer duplicateSerializer = snapshot.restoreSerializer().duplicate(); + readAndThenWriteData( + serializedData, duplicateSerializer, duplicateSerializer, testDataMatcher); + } + + // ------------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------------ + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private Path getGenerateSerializerSnapshotFilePath( + TestSpecification testSpecification) { + return Paths.get(getGenerateResourceDirectory(testSpecification) + "/serializer-snapshot"); + } + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private Path getGenerateDataFilePath( + TestSpecification testSpecification) { + return Paths.get(getGenerateResourceDirectory(testSpecification) + "/test-data"); + } + + /** Paths to use during snapshot generation, which should only use the CURRENT_VERSION. */ + private String getGenerateResourceDirectory( + TestSpecification testSpecification) { + return System.getProperty("user.dir") + + "/src/test/resources/" + + testSpecification.name + + "-" + + CURRENT_VERSION; + } + + private Path getSerializerSnapshotFilePath( + TestSpecification testSpecification) { + return Paths.get(getTestResourceDirectory(testSpecification) + "/serializer-snapshot"); + } + + private Path getTestDataFilePath( + TestSpecification testSpecification) { + return Paths.get(getTestResourceDirectory(testSpecification) + "/test-data"); + } + + private String getTestResourceDirectory( + TestSpecification testSpecification) { + return System.getProperty("user.dir") + + "/src/test/resources/" + + testSpecification.name + + "-" + + testSpecification.flinkVersion; + } + + private TypeSerializerSnapshot snapshotUnderTest( + TestSpecification testSpecification) + throws Exception { + return readSerializerSnapshot( + contentsOf(getSerializerSnapshotFilePath(testSpecification)), + testSpecification.flinkVersion); + } + + private DataInputView dataUnderTest( + TestSpecification testSpecification) { + return contentsOf(getTestDataFilePath(testSpecification)); + } + + private static void writeContentsTo(Path path, byte[] bytes) { + try { + Files.write(path, bytes); + } catch (IOException e) { + throw new RuntimeException("Failed to write to " + path, e); + } + } + + private static DataInputView contentsOf(Path path) { + try { + byte[] bytes = Files.readAllBytes(path); + return new DataInputDeserializer(bytes); + } catch (IOException e) { + throw new RuntimeException("Failed to read contents of " + path, e); + } + } + + private static void writeSerializerSnapshot( + DataOutputView out, TypeSerializer serializer, FlinkVersion flinkVersion) + throws IOException { + + if (flinkVersion.isNewerVersionThan(FlinkVersion.v1_6)) { + writeSerializerSnapshotCurrentFormat(out, serializer); + } else { + throw new UnsupportedOperationException( + "There should be no longer a need to support/use this path since Flink 1.17"); + } + } + + private static void writeSerializerSnapshotCurrentFormat( + DataOutputView out, TypeSerializer serializer) throws IOException { + + TypeSerializerSnapshotSerializationUtil.writeSerializerSnapshot( + out, serializer.snapshotConfiguration()); + } + + private static TypeSerializerSnapshot readSerializerSnapshot( + DataInputView in, FlinkVersion flinkVersion) throws IOException { + + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + checkState(flinkVersion.isNewerVersionThan(FlinkVersion.v1_6)); + return readSerializerSnapshotCurrentFormat(in, classLoader); + } + + private static TypeSerializerSnapshot readSerializerSnapshotCurrentFormat( + DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + + return TypeSerializerSnapshotSerializationUtil.readSerializerSnapshot( + in, userCodeClassLoader); + } + + private static DataInputView readAndThenWriteData( + DataInputView originalDataInput, + TypeSerializer readSerializer, + TypeSerializer writeSerializer, + Matcher testDataMatcher) + throws IOException { + + T data = readSerializer.deserialize(originalDataInput); + assertThat(data).is(HamcrestCondition.matching(testDataMatcher)); + + DataOutputSerializer out = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializer.serialize(data, out); + return new DataInputDeserializer(out.wrapAsByteBuffer()); + } + + private static TypeSerializerSnapshot writeAndThenReadSerializerSnapshot( + TypeSerializer serializer) throws IOException { + + DataOutputSerializer out = new DataOutputSerializer(INITIAL_OUTPUT_BUFFER_SIZE); + writeSerializerSnapshotCurrentFormat(out, serializer); + + DataInputDeserializer in = new DataInputDeserializer(out.wrapAsByteBuffer()); + return readSerializerSnapshotCurrentFormat( + in, Thread.currentThread().getContextClassLoader()); + } +} diff --git a/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration b/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration new file mode 100644 index 000000000..e3112c94e --- /dev/null +++ b/flink-connector-kafka/src/test/resources/META-INF/services/org.assertj.core.configuration.Configuration @@ -0,0 +1,19 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# + +org.apache.flink.KafkaAssertjConfiguration diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.11/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.12/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.13/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.14/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.15/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data index 3efe488b1..d13f52b8c 100644 Binary files a/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.16/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot new file mode 100644 index 000000000..85a71dc6b Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data new file mode 100644 index 000000000..3efe488b1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.17/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot new file mode 100644 index 000000000..85a71dc6b Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data new file mode 100644 index 000000000..3efe488b1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.18/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot new file mode 100644 index 000000000..85a71dc6b Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data new file mode 100644 index 000000000..3efe488b1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.19/test-data differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot new file mode 100644 index 000000000..85a71dc6b Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data new file mode 100644 index 000000000..3efe488b1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/context-state-serializer-1.20/test-data differ diff --git a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot b/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot deleted file mode 100644 index f3e6c74e3..000000000 Binary files a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.10-snapshot and /dev/null differ diff --git a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot b/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot deleted file mode 100644 index f80a4295e..000000000 Binary files a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.11-snapshot and /dev/null differ diff --git a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot b/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot deleted file mode 100644 index 0a22c6936..000000000 Binary files a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot and /dev/null differ diff --git a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot b/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot deleted file mode 100644 index c4f5416a4..000000000 Binary files a/flink-connector-kafka/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot and /dev/null differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot new file mode 100644 index 000000000..79ca38871 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-empty-state-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot new file mode 100644 index 000000000..08f1f7da1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.17-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot new file mode 100644 index 000000000..0fc1a5555 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-empty-state-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot new file mode 100644 index 000000000..36381498b Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.18-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-empty-state-snapshot new file mode 100644 index 000000000..d9bd99d5a Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-empty-state-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot new file mode 100644 index 000000000..ca2179cac Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.19-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot new file mode 100644 index 000000000..78ba15781 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-empty-state-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot new file mode 100644 index 000000000..995b70e58 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-consumer-migration-test-flink1.20-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot new file mode 100644 index 000000000..6d5d487df Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.17-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot new file mode 100644 index 000000000..68c9d6326 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.18-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot new file mode 100644 index 000000000..03b1a24a1 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.19-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.20-snapshot b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.20-snapshot new file mode 100644 index 000000000..a6a147cae Binary files /dev/null and b/flink-connector-kafka/src/test/resources/kafka-migration-kafka-producer-flink-1.20-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/log4j2-test.properties b/flink-connector-kafka/src/test/resources/log4j2-test.properties index 863665cf4..920652c94 100644 --- a/flink-connector-kafka/src/test/resources/log4j2-test.properties +++ b/flink-connector-kafka/src/test/resources/log4j2-test.properties @@ -27,12 +27,30 @@ appender.testlogger.target = SYSTEM_ERR appender.testlogger.layout.type = PatternLayout appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n -logger.kafka.name = kafka +# Overwrite the level for all Flink related loggers +logger.flink.name = org.apache.flink +logger.flink.level = OFF # WARN for starting debugging +logger.flinkconnector.name = org.apache.flink.connector +logger.flinkconnector.level = OFF # INFO/DEBUG for starting debugging + +# Kafka producer and consumer level +logger.kafka.name = org.apache.kafka logger.kafka.level = OFF -logger.kafka2.name = state.change -logger.kafka2.level = OFF -logger.zookeeper.name = org.apache.zookeeper -logger.zookeeper.level = OFF -logger.I0Itec.name = org.I0Itec -logger.I0Itec.level = OFF +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = OFF +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n diff --git a/flink-connector-kafka/src/test/resources/stream-metadata.yaml b/flink-connector-kafka/src/test/resources/stream-metadata.yaml new file mode 100644 index 000000000..9502351b5 --- /dev/null +++ b/flink-connector-kafka/src/test/resources/stream-metadata.yaml @@ -0,0 +1,19 @@ +- streamId: stream0 + clusterMetadataList: + - clusterId: cluster0 + bootstrapServers: bootstrap-server-0:443 + topics: + - topic0 + - topic1 + - clusterId: cluster1 + bootstrapServers: bootstrap-server-1:443 + topics: + - topic2 + - topic3 +- streamId: stream1 + clusterMetadataList: + - clusterId: cluster2 + bootstrapServers: bootstrap-server-2:443 + topics: + - topic4 + - topic5 diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot new file mode 100644 index 000000000..b95eacb63 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data new file mode 100644 index 000000000..0936509e9 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.17/test-data differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot new file mode 100644 index 000000000..b95eacb63 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data new file mode 100644 index 000000000..0936509e9 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.18/test-data differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/serializer-snapshot new file mode 100644 index 000000000..b95eacb63 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data new file mode 100644 index 000000000..0936509e9 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.19/test-data differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/serializer-snapshot b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/serializer-snapshot new file mode 100644 index 000000000..b95eacb63 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/serializer-snapshot differ diff --git a/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data new file mode 100644 index 000000000..0936509e9 Binary files /dev/null and b/flink-connector-kafka/src/test/resources/transaction-state-serializer-1.20/test-data differ diff --git a/flink-python/MANIFEST.in b/flink-python/MANIFEST.in new file mode 100644 index 000000000..3578d2dfc --- /dev/null +++ b/flink-python/MANIFEST.in @@ -0,0 +1,21 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ + +graft pyflink +global-exclude *.py[cod] __pycache__ .DS_Store + diff --git a/flink-python/README.txt b/flink-python/README.txt new file mode 100644 index 000000000..a12c13e5e --- /dev/null +++ b/flink-python/README.txt @@ -0,0 +1,14 @@ +This is official Apache Flink Kafka Python connector. + +For the latest information about Flink connector, please visit our website at: + + https://flink.apache.org + +and our GitHub Account for Kafka connector + + https://github.com/apache/flink-connector-kafka + +If you have any questions, ask on our Mailing lists: + + user@flink.apache.org + dev@flink.apache.org diff --git a/flink-python/dev/integration_test.sh b/flink-python/dev/integration_test.sh new file mode 100755 index 000000000..45f66482a --- /dev/null +++ b/flink-python/dev/integration_test.sh @@ -0,0 +1,72 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ + +function test_module() { + module="$FLINK_PYTHON_DIR/pyflink/$1" + echo "test module $module" + pytest --durations=20 ${module} $2 + if [[ $? -ne 0 ]]; then + echo "test module $module failed" + exit 1 + fi +} + +function test_all_modules() { + # test datastream module + test_module "datastream" +} + +# CURRENT_DIR is "flink-connector-kafka/flink-python/dev/" +CURRENT_DIR="$(cd "$( dirname "$0" )" && pwd)" + +# FLINK_PYTHON_DIR is "flink-connector-kafka/flink-python" +FLINK_PYTHON_DIR=$(dirname "$CURRENT_DIR") + +# FLINK_SOURCE_DIR is "flink-connector-kafka" +FLINK_SOURCE_DIR=$(dirname "$FLINK_PYTHON_DIR") + +# set the FLINK_TEST_LIB_DIR to "flink-connector-kafka" +export FLINK_TEST_LIBS="${FLINK_SOURCE_DIR}/flink-python/target/test-dependencies/*" + +# Temporarily update the installed 'pyflink_gateway_server.py' files with the new one +# Needed only until Flink 1.19 release +echo "Checking ${FLINK_SOURCE_DIR} for 'pyflink_gateway_server.py'" +find "${FLINK_SOURCE_DIR}/flink-python" -name pyflink_gateway_server.py +find "${FLINK_SOURCE_DIR}/flink-python/.tox" -name pyflink_gateway_server.py -exec cp "${FLINK_SOURCE_DIR}/flink-python/pyflink/pyflink_gateway_server.py" {} \; + +# Copy an empty flink-conf.yaml to conf/ folder, so that all Python tests on Flink 1.x can succeed. +# This needs to be changed when adding support for Flink 2.0 +echo "Checking ${FLINK_SOURCE_DIR} for 'config.yaml'" +find "${FLINK_SOURCE_DIR}/flink-python" -name config.yaml + +# For every occurrence of config.yaml (new YAML file since Flink 1.19), copy in the old flink-conf.yaml so that +# is used over the new config.yaml file. +# +# Because our intention is to copy `flink-conf.yaml` into the same directory as `config.yaml` and not replace it, +# we need to extract the directory from `{}` and then specify the target filename (`flink-conf.yaml`) explicitly. +# Unfortunately, `find`'s `-exec` doesn't directly support manipulating `{}`. So we use a slightly modified shell command +# +# `"${1}"` and `"${2}"` correspond to the first and second arguments after the shell command. +# In this case, `"${1}"` is the path to `flink-conf.yaml` and `"${2}"` is the path to each `config.yaml` found by `find`. +# `$(dirname "${2}")` extracts the directory part of the path to `config.yaml`, and then `/flink-conf.yaml` +# specifies the target filename within that directory. +find "${FLINK_SOURCE_DIR}/flink-python/.tox" -name config.yaml -exec sh -c 'cp "${1}" "$(dirname "${2}")/flink-conf.yaml"' _ "${FLINK_SOURCE_DIR}/flink-python/pyflink/flink-conf.yaml" {} \; + +# python test +test_all_modules diff --git a/flink-python/pom.xml b/flink-python/pom.xml new file mode 100644 index 000000000..dcb6117fc --- /dev/null +++ b/flink-python/pom.xml @@ -0,0 +1,222 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connector-kafka-parent + 3.4-SNAPSHOT + + + flink-connector-kafka-python + Flink : Connectors : SQL : Kafka : Python + + pom + + + + + + + org.apache.flink + flink-sql-connector-kafka + ${project.version} + + + org.apache.flink + flink-runtime + ${flink.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + + + org.apache.flink + flink-sql-avro + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + clean + clean + + run + + + + + + + + + + + + + + + + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy + + + + + org.apache.flink + flink-sql-connector-kafka + + + org.apache.flink + flink-runtime + + tests + + + org.apache.flink + flink-streaming-java + + tests + + + org.apache.flink + flink-connector-test-utils + + + org.apache.flink + flink-test-utils + + + org.apache.flink + flink-sql-avro + + + ${project.build.directory}/test-dependencies + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + junit + ${project.build.directory}/test-dependencies + + + + + + + org.codehaus.mojo + wagon-maven-plugin + 2.0.2 + + + download-install + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/install_command.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-lint + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/lint-python.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-build-wheels + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/build-wheels.sh + ${project.basedir}/dev + ${python.infra.download.skip} + + + + download-build-version-header + validate + + download-single + + + https://raw.githubusercontent.com/apache/flink-connector-shared-utils/ci_utils/python/glibc_version_fix.h + ${project.basedir}/dev + ${python.infra.download.skip} + + + + + + + diff --git a/flink-python/pyflink/datastream/connectors/kafka.py b/flink-python/pyflink/datastream/connectors/kafka.py new file mode 100644 index 000000000..062c5b2e8 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/kafka.py @@ -0,0 +1,1159 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ +import warnings +from abc import ABC, abstractmethod +from enum import Enum +from typing import Dict, Union, List, Set, Callable, Any, Optional + +from py4j.java_gateway import JavaObject, get_java_class + +from pyflink.common import DeserializationSchema, TypeInformation, typeinfo, SerializationSchema, \ + Types, Row +from pyflink.datastream.connectors import Source, Sink +from pyflink.datastream.connectors.base import DeliveryGuarantee, SupportsPreprocessing, \ + StreamTransformer +from pyflink.datastream.functions import SinkFunction, SourceFunction +from pyflink.java_gateway import get_gateway +from pyflink.util.java_utils import to_jarray, get_field, get_field_value + +__all__ = [ + 'FlinkKafkaConsumer', + 'FlinkKafkaProducer', + 'KafkaSource', + 'KafkaSourceBuilder', + 'KafkaSink', + 'KafkaSinkBuilder', + 'Semantic', + 'KafkaTopicPartition', + 'KafkaOffsetsInitializer', + 'KafkaOffsetResetStrategy', + 'KafkaRecordSerializationSchema', + 'KafkaRecordSerializationSchemaBuilder', + 'KafkaTopicSelector' +] + + +# ---- FlinkKafkaConsumer ---- + +class FlinkKafkaConsumerBase(SourceFunction, ABC): + """ + Base class of all Flink Kafka Consumer data sources. This implements the common behavior across + all kafka versions. + + The Kafka version specific behavior is defined mainly in the specific subclasses. + """ + + def __init__(self, j_flink_kafka_consumer): + super(FlinkKafkaConsumerBase, self).__init__(source_func=j_flink_kafka_consumer) + + def set_commit_offsets_on_checkpoints(self, + commit_on_checkpoints: bool) -> 'FlinkKafkaConsumerBase': + """ + Specifies whether or not the consumer should commit offsets back to kafka on checkpoints. + This setting will only have effect if checkpointing is enabled for the job. If checkpointing + isn't enabled, only the "auto.commit.enable" (for 0.8) / "enable.auto.commit" (for 0.9+) + property settings will be used. + """ + self._j_function = self._j_function \ + .setCommitOffsetsOnCheckpoints(commit_on_checkpoints) + return self + + def set_start_from_earliest(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading from the earliest offset for all partitions. This + lets the consumer ignore any committed group offsets in Zookeeper/ Kafka brokers. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromEarliest() + return self + + def set_start_from_latest(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consuer to start reading from the latest offset for all partitions. This lets + the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromLatest() + return self + + def set_start_from_timestamp(self, startup_offsets_timestamp: int) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading partitions from a specified timestamp. The specified + timestamp must be before the current timestamp. This lets the consumer ignore any committed + group offsets in Zookeeper / Kafka brokers. + + The consumer will look up the earliest offset whose timestamp is greater than or equal to + the specific timestamp from Kafka. If there's no such offset, the consumer will use the + latest offset to read data from Kafka. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + + :param startup_offsets_timestamp: timestamp for the startup offsets, as milliseconds for + epoch. + """ + self._j_function = self._j_function.setStartFromTimestamp( + startup_offsets_timestamp) + return self + + def set_start_from_group_offsets(self) -> 'FlinkKafkaConsumerBase': + """ + Specifies the consumer to start reading from any committed group offsets found in Zookeeper/ + Kafka brokers. The 'group.id' property must be set in the configuration properties. If no + offset can be found for a partition, the behaviour in 'auto.offset.reset' set in the + configuration properties will be used for the partition. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function.setStartFromGroupOffsets() + return self + + def disable_filter_restored_partitions_with_subscribed_topics(self) -> 'FlinkKafkaConsumerBase': + """ + By default, when restoring from a checkpoint / savepoint, the consumer always ignores + restored partitions that are no longer associated with the current specified topics or topic + pattern to subscribe to. + + This method does not affect where partitions are read from when the consumer is restored + from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + savepoint, only the offsets in the restored state will be used. + """ + self._j_function = self._j_function \ + .disableFilterRestoredPartitionsWithSubscribedTopics() + return self + + def get_produced_type(self) -> TypeInformation: + return typeinfo._from_java_type(self._j_function.getProducedType()) + + +def _get_kafka_consumer(topics, properties, deserialization_schema, j_consumer_clz): + if not isinstance(topics, list): + topics = [topics] + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in properties.items(): + j_properties.setProperty(key, value) + + j_flink_kafka_consumer = j_consumer_clz(topics, + deserialization_schema._j_deserialization_schema, + j_properties) + return j_flink_kafka_consumer + + +class FlinkKafkaConsumer(FlinkKafkaConsumerBase): + """ + The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from + Apache Kafka. The consumer can run in multiple parallel instances, each of which will + pull data from one or more Kafka partitions. + + The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost + during a failure, and that the computation processes elements 'exactly once. (These guarantees + naturally assume that Kafka itself does not lose any data.) + + Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. + The offsets committed to Kafka / Zookeeper are only to bring the outside view of progress in + sync with Flink's view of the progress. That way, monitoring and other jobs can get a view of + how far the Flink Kafka consumer has consumed a topic. + + Please refer to Kafka's documentation for the available configuration properties: + http://kafka.apache.org/documentation.html#newconsumerconfigs + """ + + def __init__(self, topics: Union[str, List[str]], deserialization_schema: DeserializationSchema, + properties: Dict): + """ + Creates a new Kafka streaming source consumer for Kafka 0.10.x. + + This constructor allows passing multiple topics to the consumer. + + :param topics: The Kafka topics to read from. + :param deserialization_schema: The de-/serializer used to convert between Kafka's byte + messages and Flink's objects. + :param properties: The properties that are used to configure both the fetcher and the offset + handler. + """ + + warnings.warn("Deprecated in 1.16. Use KafkaSource instead.", DeprecationWarning) + JFlinkKafkaConsumer = get_gateway().jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer + j_flink_kafka_consumer = _get_kafka_consumer(topics, properties, deserialization_schema, + JFlinkKafkaConsumer) + super(FlinkKafkaConsumer, self).__init__(j_flink_kafka_consumer=j_flink_kafka_consumer) + + +# ---- FlinkKafkaProducer ---- + + +class Semantic(Enum): + """ + Semantics that can be chosen. + + :data: `EXACTLY_ONCE`: + + The Flink producer will write all messages in a Kafka transaction that will be committed to + the Kafka on a checkpoint. In this mode FlinkKafkaProducer sets up a pool of + FlinkKafkaProducer. Between each checkpoint there is created new Kafka transaction, which is + being committed on FlinkKafkaProducer.notifyCheckpointComplete(long). If checkpoint + complete notifications are running late, FlinkKafkaProducer can run out of + FlinkKafkaProducers in the pool. In that case any subsequent FlinkKafkaProducer.snapshot- + State() requests will fail and the FlinkKafkaProducer will keep using the + FlinkKafkaProducer from previous checkpoint. To decrease chances of failing checkpoints + there are four options: + + 1. decrease number of max concurrent checkpoints + 2. make checkpoints mre reliable (so that they complete faster) + 3. increase delay between checkpoints + 4. increase size of FlinkKafkaProducers pool + + :data: `AT_LEAST_ONCE`: + + The Flink producer will wait for all outstanding messages in the Kafka buffers to be + acknowledged by the Kafka producer on a checkpoint. + + :data: `NONE`: + + Means that nothing will be guaranteed. Messages can be lost and/or duplicated in case of + failure. + + """ + + EXACTLY_ONCE = 0, + AT_LEAST_ONCE = 1, + NONE = 2 + + def _to_j_semantic(self): + JSemantic = get_gateway().jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic + return getattr(JSemantic, self.name) + + +class FlinkKafkaProducerBase(SinkFunction, ABC): + """ + Flink Sink to produce data into a Kafka topic. + + Please note that this producer provides at-least-once reliability guarantees when checkpoints + are enabled and set_flush_on_checkpoint(True) is set. Otherwise, the producer doesn;t provid any + reliability guarantees. + """ + + def __init__(self, j_flink_kafka_producer): + super(FlinkKafkaProducerBase, self).__init__(sink_func=j_flink_kafka_producer) + + def set_log_failures_only(self, log_failures_only: bool) -> 'FlinkKafkaProducerBase': + """ + Defines whether the producer should fail on errors, or only log them. If this is set to + true, then exceptions will be only logged, if set to false, exceptions will be eventually + thrown and cause the streaming program to fail (and enter recovery). + + :param log_failures_only: The flag to indicate logging-only on exceptions. + """ + self._j_function.setLogFailuresOnly(log_failures_only) + return self + + def set_flush_on_checkpoint(self, flush_on_checkpoint: bool) -> 'FlinkKafkaProducerBase': + """ + If set to true, the Flink producer will wait for all outstanding messages in the Kafka + buffers to be acknowledged by the Kafka producer on a checkpoint. + + This way, the producer can guarantee that messages in the Kafka buffers are part of the + checkpoint. + + :param flush_on_checkpoint: Flag indicating the flush mode (true = flush on checkpoint) + """ + self._j_function.setFlushOnCheckpoint(flush_on_checkpoint) + return self + + def set_write_timestamp_to_kafka(self, + write_timestamp_to_kafka: bool) -> 'FlinkKafkaProducerBase': + """ + If set to true, Flink will write the (event time) timestamp attached to each record into + Kafka. Timestamps must be positive for Kafka to accept them. + + :param write_timestamp_to_kafka: Flag indicating if Flink's internal timestamps are written + to Kafka. + """ + self._j_function.setWriteTimestampToKafka(write_timestamp_to_kafka) + return self + + +class FlinkKafkaProducer(FlinkKafkaProducerBase): + """ + Flink Sink to produce data into a Kafka topic. By + default producer will use AT_LEAST_ONCE semantic. Before using EXACTLY_ONCE please refer to + Flink's Kafka connector documentation. + """ + + def __init__(self, topic: str, serialization_schema: SerializationSchema, + producer_config: Dict, kafka_producer_pool_size: int = 5, + semantic=Semantic.AT_LEAST_ONCE): + """ + Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to the topic. + + Using this constructor, the default FlinkFixedPartitioner will be used as the partitioner. + This default partitioner maps each sink subtask to a single Kafka partition (i.e. all + records received by a sink subtask will end up in the same Kafka partition). + + :param topic: ID of the Kafka topic. + :param serialization_schema: User defined key-less serialization schema. + :param producer_config: Properties with the producer configuration. + """ + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in producer_config.items(): + j_properties.setProperty(key, value) + + JFlinkKafkaProducer = gateway.jvm \ + .org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer + + j_flink_kafka_producer = JFlinkKafkaProducer( + topic, serialization_schema._j_serialization_schema, j_properties, None, + semantic._to_j_semantic(), kafka_producer_pool_size) + super(FlinkKafkaProducer, self).__init__(j_flink_kafka_producer=j_flink_kafka_producer) + + def ignore_failures_after_transaction_timeout(self) -> 'FlinkKafkaProducer': + """ + Disables the propagation of exceptions thrown when committing presumably timed out Kafka + transactions during recovery of the job. If a Kafka transaction is timed out, a commit will + never be successful. Hence, use this feature to avoid recovery loops of the Job. Exceptions + will still be logged to inform the user that data loss might have occurred. + + Note that we use the System.currentTimeMillis() to track the age of a transaction. Moreover, + only exceptions thrown during the recovery are caught, i.e., the producer will attempt at + least one commit of the transaction before giving up. + + :return: This FlinkKafkaProducer. + """ + self._j_function.ignoreFailuresAfterTransactionTimeout() + return self + + +# ---- KafkaSource ---- + + +class KafkaSource(Source): + """ + The Source implementation of Kafka. Please use a :class:`KafkaSourceBuilder` to construct a + :class:`KafkaSource`. The following example shows how to create a KafkaSource emitting records + of String type. + + :: + + >>> source = KafkaSource \\ + ... .builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_group_id('MY_GROUP') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .set_starting_offsets(KafkaOffsetsInitializer.earliest()) \\ + ... .build() + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_kafka_source: JavaObject): + super().__init__(j_kafka_source) + + @staticmethod + def builder() -> 'KafkaSourceBuilder': + """ + Get a kafkaSourceBuilder to build a :class:`KafkaSource`. + + :return: a Kafka source builder. + """ + return KafkaSourceBuilder() + + +class KafkaSourceBuilder(object): + """ + The builder class for :class:`KafkaSource` to make it easier for the users to construct a + :class:`KafkaSource`. + + The following example shows the minimum setup to create a KafkaSource that reads the String + values from a Kafka topic. + + :: + + >>> source = KafkaSource.builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .build() + + The bootstrap servers, topics/partitions to consume, and the record deserializer are required + fields that must be set. + + To specify the starting offsets of the KafkaSource, one can call :meth:`set_starting_offsets`. + + By default, the KafkaSource runs in an CONTINUOUS_UNBOUNDED mode and never stops until the Flink + job is canceled or fails. To let the KafkaSource run in CONTINUOUS_UNBOUNDED but stops at some + given offsets, one can call :meth:`set_stopping_offsets`. For example the following KafkaSource + stops after it consumes up to the latest partition offsets at the point when the Flink started. + + :: + + >>> source = KafkaSource.builder() \\ + ... .set_bootstrap_servers('MY_BOOTSTRAP_SERVERS') \\ + ... .set_topics('TOPIC1', 'TOPIC2') \\ + ... .set_value_only_deserializer(SimpleStringSchema()) \\ + ... .set_unbounded(KafkaOffsetsInitializer.latest()) \\ + ... .build() + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + self._j_builder = get_gateway().jvm.org.apache.flink.connector.kafka.source \ + .KafkaSource.builder() + + def build(self) -> 'KafkaSource': + return KafkaSource(self._j_builder.build()) + + def set_bootstrap_servers(self, bootstrap_servers: str) -> 'KafkaSourceBuilder': + """ + Sets the bootstrap servers for the KafkaConsumer of the KafkaSource. + + :param bootstrap_servers: the bootstrap servers of the Kafka cluster. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setBootstrapServers(bootstrap_servers) + return self + + def set_group_id(self, group_id: str) -> 'KafkaSourceBuilder': + """ + Sets the consumer group id of the KafkaSource. + + :param group_id: the group id of the KafkaSource. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setGroupId(group_id) + return self + + def set_topics(self, *topics: str) -> 'KafkaSourceBuilder': + """ + Set a list of topics the KafkaSource should consume from. All the topics in the list should + have existed in the Kafka cluster. Otherwise, an exception will be thrown. To allow some + topics to be created lazily, please use :meth:`set_topic_pattern` instead. + + :param topics: the list of topics to consume from. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setTopics(to_jarray(get_gateway().jvm.java.lang.String, topics)) + return self + + def set_topic_pattern(self, topic_pattern: str) -> 'KafkaSourceBuilder': + """ + Set a topic pattern to consume from use the java Pattern. For grammar, check out + `JavaDoc `_ . + + :param topic_pattern: the pattern of the topic name to consume from. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setTopicPattern(get_gateway().jvm.java.util.regex + .Pattern.compile(topic_pattern)) + return self + + def set_partitions(self, partitions: Set['KafkaTopicPartition']) -> 'KafkaSourceBuilder': + """ + Set a set of partitions to consume from. + + Example: + :: + + >>> KafkaSource.builder().set_partitions({ + ... KafkaTopicPartition('TOPIC1', 0), + ... KafkaTopicPartition('TOPIC1', 1), + ... }) + + :param partitions: the set of partitions to consume from. + :return: this KafkaSourceBuilder. + """ + j_set = get_gateway().jvm.java.util.HashSet() + for tp in partitions: + j_set.add(tp._to_j_topic_partition()) + self._j_builder.setPartitions(j_set) + return self + + def set_starting_offsets(self, starting_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + Specify from which offsets the KafkaSource should start consume from by providing an + :class:`KafkaOffsetsInitializer`. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.earliest` - starting from the earliest offsets. This is + also the default offset initializer of the KafkaSource for starting offsets. + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param starting_offsets_initializer: the :class:`KafkaOffsetsInitializer` setting the + starting offsets for the Source. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setStartingOffsets(starting_offsets_initializer._j_initializer) + return self + + def set_unbounded(self, stopping_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never + stops until the Flink job fails or is canceled. To let the KafkaSource run as a streaming + source but still stops at some point, one can set an :class:`KafkaOffsetsInitializer` + to specify the stopping offsets for each partition. When all the partitions have reached + their stopping offsets, the KafkaSource will then exit. + + This method is different from :meth:`set_bounded` that after setting the stopping offsets + with this method, KafkaSource will still be CONTINUOUS_UNBOUNDED even though it will stop at + the stopping offsets specified by the stopping offset initializer. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param stopping_offsets_initializer: the :class:`KafkaOffsetsInitializer` to specify the + stopping offsets. + :return: this KafkaSourceBuilder + """ + self._j_builder.setUnbounded(stopping_offsets_initializer._j_initializer) + return self + + def set_bounded(self, stopping_offsets_initializer: 'KafkaOffsetsInitializer') \ + -> 'KafkaSourceBuilder': + """ + By default, the KafkaSource is set to run in CONTINUOUS_UNBOUNDED manner and thus never + stops until the Flink job fails or is canceled. To let the KafkaSource run in BOUNDED manner + and stop at some point, one can set an :class:`KafkaOffsetsInitializer` to specify the + stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the KafkaSource will then exit. + + This method is different from :meth:`set_unbounded` that after setting the stopping offsets + with this method, :meth:`KafkaSource.get_boundedness` will return BOUNDED instead of + CONTINUOUS_UNBOUNDED. + + The following :class:`KafkaOffsetsInitializer` s are commonly used and provided out of the + box. Currently, customized offset initializer is not supported in PyFlink. + + * :meth:`KafkaOffsetsInitializer.latest` - starting from the latest offsets. + * :meth:`KafkaOffsetsInitializer.committedOffsets` - starting from the committed offsets of + the consumer group. If there is no committed offsets, starting from the offsets + specified by the :class:`KafkaOffsetResetStrategy`. + * :meth:`KafkaOffsetsInitializer.offsets` - starting from the specified offsets for each + partition. + * :meth:`KafkaOffsetsInitializer.timestamp` - starting from the specified timestamp for each + partition. Note that the guarantee here is that all the records in Kafka whose timestamp + is greater than the given starting timestamp will be consumed. However, it is possible + that some consumer records whose timestamp is smaller than the given starting timestamp + are also consumed. + + :param stopping_offsets_initializer: the :class:`KafkaOffsetsInitializer` to specify the + stopping offsets. + :return: this KafkaSourceBuilder + """ + self._j_builder.setBounded(stopping_offsets_initializer._j_initializer) + return self + + def set_value_only_deserializer(self, deserialization_schema: DeserializationSchema) \ + -> 'KafkaSourceBuilder': + """ + Sets the :class:`~pyflink.common.serialization.DeserializationSchema` for deserializing the + value of Kafka's ConsumerRecord. The other information (e.g. key) in a ConsumerRecord will + be ignored. + + :param deserialization_schema: the :class:`DeserializationSchema` to use for + deserialization. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setValueOnlyDeserializer(deserialization_schema._j_deserialization_schema) + return self + + def set_client_id_prefix(self, prefix: str) -> 'KafkaSourceBuilder': + """ + Sets the client id prefix of this KafkaSource. + + :param prefix: the client id prefix to use for this KafkaSource. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setClientIdPrefix(prefix) + return self + + def set_property(self, key: str, value: str) -> 'KafkaSourceBuilder': + """ + Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found + in ConsumerConfig and KafkaSourceOptions. + + Note that the following keys will be overridden by the builder when the KafkaSource is + created. + + * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by + :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default + :meth:`KafkaOffsetsInitializer.earliest`. + * ``partition.discovery.interval.ms`` is overridden to -1 when :meth:`set_bounded` has been + invoked. + + :param key: the key of the property. + :param value: the value of the property. + :return: this KafkaSourceBuilder. + """ + self._j_builder.setProperty(key, value) + return self + + def set_properties(self, props: Dict) -> 'KafkaSourceBuilder': + """ + Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found + in ConsumerConfig and KafkaSourceOptions. + + Note that the following keys will be overridden by the builder when the KafkaSource is + created. + + * ``auto.offset.reset.strategy`` is overridden by AutoOffsetResetStrategy returned by + :class:`KafkaOffsetsInitializer` for the starting offsets, which is by default + :meth:`KafkaOffsetsInitializer.earliest`. + * ``partition.discovery.interval.ms`` is overridden to -1 when :meth:`set_bounded` has been + invoked. + * ``client.id`` is overridden to "client.id.prefix-RANDOM_LONG", or "group.id-RANDOM_LONG" + if the client id prefix is not set. + + :param props: the properties to set for the KafkaSource. + :return: this KafkaSourceBuilder. + """ + gateway = get_gateway() + j_properties = gateway.jvm.java.util.Properties() + for key, value in props.items(): + j_properties.setProperty(key, value) + self._j_builder.setProperties(j_properties) + return self + + +class KafkaTopicPartition(object): + """ + Corresponding to Java ``org.apache.kafka.common.TopicPartition`` class. + + Example: + :: + + >>> topic_partition = KafkaTopicPartition('TOPIC1', 0) + + .. versionadded:: 1.16.0 + """ + + def __init__(self, topic: str, partition: int): + self._topic = topic + self._partition = partition + + def _to_j_topic_partition(self): + jvm = get_gateway().jvm + return jvm.org.apache.flink.kafka.shaded.org.apache.kafka.common.TopicPartition( + self._topic, self._partition) + + def __eq__(self, other): + if not isinstance(other, KafkaTopicPartition): + return False + return self._topic == other._topic and self._partition == other._partition + + def __hash__(self): + return 31 * (31 + self._partition) + hash(self._topic) + + +class KafkaOffsetResetStrategy(Enum): + """ + Corresponding to Java ``org.apache.kafka.client.consumer.OffsetResetStrategy`` class. + + .. versionadded:: 1.16.0 + """ + + LATEST = 0 + EARLIEST = 1 + NONE = 2 + + def _to_j_offset_reset_strategy(self): + JOffsetResetStrategy = get_gateway().jvm.org.apache.flink.kafka.shaded.org.apache.kafka.\ + clients.consumer.OffsetResetStrategy + return getattr(JOffsetResetStrategy, self.name) + + +class KafkaOffsetsInitializer(object): + """ + An interface for users to specify the starting / stopping offset of a KafkaPartitionSplit. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_initializer: JavaObject): + self._j_initializer = j_initializer + + @staticmethod + def committed_offsets( + offset_reset_strategy: 'KafkaOffsetResetStrategy' = KafkaOffsetResetStrategy.NONE) -> \ + 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the committed + offsets. An exception will be thrown at runtime if there is no committed offsets. + + An optional :class:`KafkaOffsetResetStrategy` can be specified to initialize the offsets if + the committed offsets does not exist. + + :param offset_reset_strategy: the offset reset strategy to use when the committed offsets do + not exist. + :return: an offset initializer which initialize the offsets to the committed offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source.\ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.committedOffsets( + offset_reset_strategy._to_j_offset_reset_strategy())) + + @staticmethod + def timestamp(timestamp: int) -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets in each partition so + that the initialized offset is the offset of the first record whose record timestamp is + greater than or equals the give timestamp. + + :param timestamp: the timestamp to start the consumption. + :return: an :class:`OffsetsInitializer` which initializes the offsets based on the given + timestamp. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.timestamp(timestamp)) + + @staticmethod + def earliest() -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the earliest + available offsets of each partition. + + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the earliest + available offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.earliest()) + + @staticmethod + def latest() -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the latest offsets + of each partition. + + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the latest + offsets. + """ + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.latest()) + + @staticmethod + def offsets(offsets: Dict['KafkaTopicPartition', int], + offset_reset_strategy: 'KafkaOffsetResetStrategy' = + KafkaOffsetResetStrategy.EARLIEST) -> 'KafkaOffsetsInitializer': + """ + Get an :class:`KafkaOffsetsInitializer` which initializes the offsets to the specified + offsets. + + An optional :class:`KafkaOffsetResetStrategy` can be specified to initialize the offsets in + case the specified offset is out of range. + + Example: + :: + + >>> KafkaOffsetsInitializer.offsets({ + ... KafkaTopicPartition('TOPIC1', 0): 0, + ... KafkaTopicPartition('TOPIC1', 1): 10000 + ... }, KafkaOffsetResetStrategy.EARLIEST) + + :param offsets: the specified offsets for each partition. + :param offset_reset_strategy: the :class:`KafkaOffsetResetStrategy` to use when the + specified offset is out of range. + :return: an :class:`KafkaOffsetsInitializer` which initializes the offsets to the specified + offsets. + """ + jvm = get_gateway().jvm + j_map_wrapper = jvm.org.apache.flink.python.util.HashMapWrapper( + None, get_java_class(jvm.Long)) + for tp, offset in offsets.items(): + j_map_wrapper.put(tp._to_j_topic_partition(), offset) + + JOffsetsInitializer = get_gateway().jvm.org.apache.flink.connector.kafka.source. \ + enumerator.initializer.OffsetsInitializer + return KafkaOffsetsInitializer(JOffsetsInitializer.offsets( + j_map_wrapper.asMap(), offset_reset_strategy._to_j_offset_reset_strategy())) + + +class KafkaSink(Sink, SupportsPreprocessing): + """ + Flink Sink to produce data into a Kafka topic. The sink supports all delivery guarantees + described by :class:`DeliveryGuarantee`. + + * :attr:`DeliveryGuarantee.NONE` does not provide any guarantees: messages may be lost in case + of issues on the Kafka broker and messages may be duplicated in case of a Flink failure. + * :attr:`DeliveryGuarantee.AT_LEAST_ONCE` the sink will wait for all outstanding records in the + Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. No messages will be + lost in case of any issue with the Kafka brokers but messages may be duplicated when Flink + restarts. + * :attr:`DeliveryGuarantee.EXACTLY_ONCE`: In this mode the KafkaSink will write all messages in + a Kafka transaction that will be committed to Kafka on a checkpoint. Thus, if the consumer + reads only committed data (see Kafka consumer config ``isolation.level``), no duplicates + will be seen in case of a Flink restart. However, this delays record writing effectively + until a checkpoint is written, so adjust the checkpoint duration accordingly. Please ensure + that you use unique transactional id prefixes across your applications running on the same + Kafka cluster such that multiple running jobs do not interfere in their transactions! + Additionally, it is highly recommended to tweak Kafka transaction timeout (link) >> maximum + checkpoint duration + maximum restart duration or data loss may happen when Kafka expires an + uncommitted transaction. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_kafka_sink, transformer: Optional[StreamTransformer] = None): + super().__init__(j_kafka_sink) + self._transformer = transformer + + @staticmethod + def builder() -> 'KafkaSinkBuilder': + """ + Create a :class:`KafkaSinkBuilder` to construct :class:`KafkaSink`. + """ + return KafkaSinkBuilder() + + def get_transformer(self) -> Optional[StreamTransformer]: + return self._transformer + + +class KafkaSinkBuilder(object): + """ + Builder to construct :class:`KafkaSink`. + + The following example shows the minimum setup to create a KafkaSink that writes String values + to a Kafka topic. + + :: + + >>> record_serializer = KafkaRecordSerializationSchema.builder() \\ + ... .set_topic(MY_SINK_TOPIC) \\ + ... .set_value_serialization_schema(SimpleStringSchema()) \\ + ... .build() + >>> sink = KafkaSink.builder() \\ + ... .set_bootstrap_servers(MY_BOOTSTRAP_SERVERS) \\ + ... .set_record_serializer(record_serializer) \\ + ... .build() + + One can also configure different :class:`DeliveryGuarantee` by using + :meth:`set_delivery_guarantee` but keep in mind when using + :attr:`DeliveryGuarantee.EXACTLY_ONCE`, one must set the transactional id prefix + :meth:`set_transactional_id_prefix`. + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + jvm = get_gateway().jvm + self._j_builder = jvm.org.apache.flink.connector.kafka.sink.KafkaSink.builder() + self._preprocessing = None + + def build(self) -> 'KafkaSink': + """ + Constructs the :class:`KafkaSink` with the configured properties. + """ + return KafkaSink(self._j_builder.build(), self._preprocessing) + + def set_bootstrap_servers(self, bootstrap_servers: str) -> 'KafkaSinkBuilder': + """ + Sets the Kafka bootstrap servers. + + :param bootstrap_servers: A comma separated list of valid URIs to reach the Kafka broker. + """ + self._j_builder.setBootstrapServers(bootstrap_servers) + return self + + def set_delivery_guarantee(self, delivery_guarantee: DeliveryGuarantee) -> 'KafkaSinkBuilder': + """ + Sets the wanted :class:`DeliveryGuarantee`. The default delivery guarantee is + :attr:`DeliveryGuarantee.NONE`. + + :param delivery_guarantee: The wanted :class:`DeliveryGuarantee`. + """ + self._j_builder.setDeliveryGuarantee(delivery_guarantee._to_j_delivery_guarantee()) + return self + + def set_transactional_id_prefix(self, transactional_id_prefix: str) -> 'KafkaSinkBuilder': + """ + Sets the prefix for all created transactionalIds if :attr:`DeliveryGuarantee.EXACTLY_ONCE` + is configured. + + It is mandatory to always set this value with :attr:`DeliveryGuarantee.EXACTLY_ONCE` to + prevent corrupted transactions if multiple jobs using the KafkaSink run against the same + Kafka Cluster. The default prefix is ``"kafka-sink"``. + + The size of the prefix is capped by MAXIMUM_PREFIX_BYTES (6400) formatted with UTF-8. + + It is important to keep the prefix stable across application restarts. If the prefix changes + it might happen that lingering transactions are not correctly aborted and newly written + messages are not immediately consumable until transactions timeout. + + :param transactional_id_prefix: The transactional id prefix. + """ + self._j_builder.setTransactionalIdPrefix(transactional_id_prefix) + return self + + def set_record_serializer(self, record_serializer: 'KafkaRecordSerializationSchema') \ + -> 'KafkaSinkBuilder': + """ + Sets the :class:`KafkaRecordSerializationSchema` that transforms incoming records to kafka + producer records. + + :param record_serializer: The :class:`KafkaRecordSerializationSchema`. + """ + # NOTE: If topic selector is a generated first-column selector, do extra preprocessing + j_topic_selector = get_field_value(record_serializer._j_serialization_schema, + 'topicSelector') + if ( + j_topic_selector.getClass().getCanonicalName() == + 'org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchemaBuilder.' + 'CachingTopicSelector' + ) and ( + get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + is not None and + (get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + .startswith('com.sun.proxy') or + get_field_value(j_topic_selector, 'topicSelector').getClass().getCanonicalName() + .startswith('jdk.proxy')) + ): + record_serializer._wrap_serialization_schema() + self._preprocessing = record_serializer._build_preprocessing() + + self._j_builder.setRecordSerializer(record_serializer._j_serialization_schema) + return self + + def set_property(self, key: str, value: str) -> 'KafkaSinkBuilder': + """ + Sets kafka producer config. + + :param key: Kafka producer config key. + :param value: Kafka producer config value. + """ + self._j_builder.setProperty(key, value) + return self + + +class KafkaRecordSerializationSchema(SerializationSchema): + """ + A serialization schema which defines how to convert the stream record to kafka producer record. + + .. versionadded:: 1.16.0 + """ + + def __init__(self, j_serialization_schema, + topic_selector: Optional['KafkaTopicSelector'] = None): + super().__init__(j_serialization_schema) + self._topic_selector = topic_selector + + @staticmethod + def builder() -> 'KafkaRecordSerializationSchemaBuilder': + """ + Creates a default schema builder to provide common building blocks i.e. key serialization, + value serialization, topic selection. + """ + return KafkaRecordSerializationSchemaBuilder() + + def _wrap_serialization_schema(self): + jvm = get_gateway().jvm + + def _wrap_schema(field_name): + j_schema_field = get_field(self._j_serialization_schema.getClass(), field_name) + if j_schema_field.get(self._j_serialization_schema) is not None: + j_schema_field.set( + self._j_serialization_schema, + jvm.org.apache.flink.python.util.PythonConnectorUtils + .SecondColumnSerializationSchema( + j_schema_field.get(self._j_serialization_schema) + ) + ) + + _wrap_schema('keySerializationSchema') + _wrap_schema('valueSerializationSchema') + + def _build_preprocessing(self) -> StreamTransformer: + class SelectTopicTransformer(StreamTransformer): + + def __init__(self, topic_selector: KafkaTopicSelector): + self._topic_selector = topic_selector + + def apply(self, ds): + output_type = Types.ROW([Types.STRING(), ds.get_type()]) + return ds.map(lambda v: Row(self._topic_selector.apply(v), v), + output_type=output_type) + + return SelectTopicTransformer(self._topic_selector) + + +class KafkaRecordSerializationSchemaBuilder(object): + """ + Builder to construct :class:`KafkaRecordSerializationSchema`. + + Example: + :: + + >>> KafkaRecordSerializationSchema.builder() \\ + ... .set_topic('topic') \\ + ... .set_key_serialization_schema(SimpleStringSchema()) \\ + ... .set_value_serialization_schema(SimpleStringSchema()) \\ + ... .build() + + And the sink topic can be calculated dynamically from each record: + :: + + >>> KafkaRecordSerializationSchema.builder() \\ + ... .set_topic_selector(lambda row: 'topic-' + row['category']) \\ + ... .set_value_serialization_schema( + ... JsonRowSerializationSchema.builder().with_type_info(ROW_TYPE).build()) \\ + ... .build() + + It is necessary to configure exactly one serialization method for the value and a topic. + + .. versionadded:: 1.16.0 + """ + + def __init__(self): + jvm = get_gateway().jvm + self._j_builder = jvm.org.apache.flink.connector.kafka.sink \ + .KafkaRecordSerializationSchemaBuilder() + self._fixed_topic = True # type: bool + self._topic_selector = None # type: Optional[KafkaTopicSelector] + self._key_serialization_schema = None # type: Optional[SerializationSchema] + self._value_serialization_schema = None # type: Optional[SerializationSchema] + + def build(self) -> 'KafkaRecordSerializationSchema': + """ + Constructs the :class:`KafkaRecordSerializationSchemaBuilder` with the configured + properties. + """ + if self._fixed_topic: + return KafkaRecordSerializationSchema(self._j_builder.build()) + else: + return KafkaRecordSerializationSchema(self._j_builder.build(), self._topic_selector) + + def set_topic(self, topic: str) -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a fixed topic which used as destination for all records. + + :param topic: The fixed topic. + """ + self._j_builder.setTopic(topic) + self._fixed_topic = True + return self + + def set_topic_selector(self, topic_selector: Union[Callable[[Any], str], 'KafkaTopicSelector'])\ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a topic selector which computes the target topic for every incoming record. + + :param topic_selector: A :class:`KafkaTopicSelector` implementation or a function that + consumes each incoming record and return the topic string. + """ + if not isinstance(topic_selector, KafkaTopicSelector) and not callable(topic_selector): + raise TypeError('topic_selector must be KafkaTopicSelector or a callable') + if not isinstance(topic_selector, KafkaTopicSelector): + class TopicSelectorFunctionAdapter(KafkaTopicSelector): + + def __init__(self, f: Callable[[Any], str]): + self._f = f + + def apply(self, data) -> str: + return self._f(data) + + topic_selector = TopicSelectorFunctionAdapter(topic_selector) + + jvm = get_gateway().jvm + self._j_builder.setTopicSelector( + jvm.org.apache.flink.python.util.PythonConnectorUtils.createFirstColumnTopicSelector( + get_java_class(jvm.org.apache.flink.connector.kafka.sink.TopicSelector) + ) + ) + self._fixed_topic = False + self._topic_selector = topic_selector + return self + + def set_key_serialization_schema(self, key_serialization_schema: SerializationSchema) \ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a :class:`SerializationSchema` which is used to serialize the incoming element to the + key of the producer record. The key serialization is optional, if not set, the key of the + producer record will be null. + + :param key_serialization_schema: The :class:`SerializationSchema` to serialize each incoming + record as the key of producer record. + """ + self._key_serialization_schema = key_serialization_schema + self._j_builder.setKeySerializationSchema(key_serialization_schema._j_serialization_schema) + return self + + def set_value_serialization_schema(self, value_serialization_schema: SerializationSchema) \ + -> 'KafkaRecordSerializationSchemaBuilder': + """ + Sets a :class:`SerializationSchema` which is used to serialize the incoming element to the + value of the producer record. The value serialization is required. + + :param value_serialization_schema: The :class:`SerializationSchema` to serialize each data + record as the key of producer record. + """ + self._value_serialization_schema = value_serialization_schema + self._j_builder.setValueSerializationSchema( + value_serialization_schema._j_serialization_schema) + return self + + +class KafkaTopicSelector(ABC): + """ + Select topic for an incoming record + + .. versionadded:: 1.16.0 + """ + + @abstractmethod + def apply(self, data) -> str: + pass diff --git a/flink-python/pyflink/datastream/connectors/tests/test_kafka.py b/flink-python/pyflink/datastream/connectors/tests/test_kafka.py new file mode 100644 index 000000000..dea06b3e0 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/tests/test_kafka.py @@ -0,0 +1,669 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ +import json +from typing import Dict + +import pyflink.datastream.data_stream as data_stream +from pyflink.common import typeinfo + +from pyflink.common.configuration import Configuration +from pyflink.common.serialization import SimpleStringSchema, DeserializationSchema +from pyflink.common.typeinfo import Types +from pyflink.common.types import Row +from pyflink.common.watermark_strategy import WatermarkStrategy +from pyflink.datastream.connectors.base import DeliveryGuarantee +from pyflink.datastream.connectors.kafka import KafkaSource, KafkaTopicPartition, \ + KafkaOffsetsInitializer, KafkaOffsetResetStrategy, KafkaRecordSerializationSchema, KafkaSink, \ + FlinkKafkaProducer, FlinkKafkaConsumer +from pyflink.datastream.formats.avro import AvroRowDeserializationSchema, AvroRowSerializationSchema +from pyflink.datastream.formats.csv import CsvRowDeserializationSchema, CsvRowSerializationSchema +from pyflink.datastream.formats.json import JsonRowDeserializationSchema, JsonRowSerializationSchema +from pyflink.java_gateway import get_gateway +from pyflink.testing.test_case_utils import ( + PyFlinkStreamingTestCase, + PyFlinkTestCase, + invoke_java_object_method, + to_java_data_structure, +) +from pyflink.util.java_utils import to_jarray, is_instance_of, get_field_value + + +class KafkaSourceTests(PyFlinkStreamingTestCase): + + def test_legacy_kafka_connector(self): + source_topic = 'test_source_topic' + sink_topic = 'test_sink_topic' + props = {'bootstrap.servers': 'localhost:9092', 'group.id': 'test_group'} + type_info = Types.ROW([Types.INT(), Types.STRING()]) + + # Test for kafka consumer + deserialization_schema = JsonRowDeserializationSchema.builder() \ + .type_info(type_info=type_info).build() + + flink_kafka_consumer = FlinkKafkaConsumer(source_topic, deserialization_schema, props) + flink_kafka_consumer.set_start_from_earliest() + flink_kafka_consumer.set_commit_offsets_on_checkpoints(True) + + j_properties = get_field_value(flink_kafka_consumer.get_java_function(), 'properties') + self.assertEqual('localhost:9092', j_properties.getProperty('bootstrap.servers')) + self.assertEqual('test_group', j_properties.getProperty('group.id')) + self.assertTrue(get_field_value(flink_kafka_consumer.get_java_function(), + 'enableCommitOnCheckpoints')) + j_start_up_mode = get_field_value(flink_kafka_consumer.get_java_function(), 'startupMode') + + j_deserializer = get_field_value(flink_kafka_consumer.get_java_function(), 'deserializer') + j_deserialize_type_info = invoke_java_object_method(j_deserializer, "getProducedType") + deserialize_type_info = typeinfo._from_java_type(j_deserialize_type_info) + self.assertTrue(deserialize_type_info == type_info) + self.assertTrue(j_start_up_mode.equals(get_gateway().jvm + .org.apache.flink.streaming.connectors + .kafka.config.StartupMode.EARLIEST)) + j_topic_desc = get_field_value(flink_kafka_consumer.get_java_function(), + 'topicsDescriptor') + j_topics = invoke_java_object_method(j_topic_desc, 'getFixedTopics') + self.assertEqual(['test_source_topic'], list(j_topics)) + + # Test for kafka producer + serialization_schema = JsonRowSerializationSchema.builder().with_type_info(type_info) \ + .build() + flink_kafka_producer = FlinkKafkaProducer(sink_topic, serialization_schema, props) + flink_kafka_producer.set_write_timestamp_to_kafka(False) + + j_producer_config = get_field_value(flink_kafka_producer.get_java_function(), + 'producerConfig') + self.assertEqual('localhost:9092', j_producer_config.getProperty('bootstrap.servers')) + self.assertEqual('test_group', j_producer_config.getProperty('group.id')) + self.assertFalse(get_field_value(flink_kafka_producer.get_java_function(), + 'writeTimestampToKafka')) + + def test_compiling(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + + ds = self.env.from_source(source=source, + watermark_strategy=WatermarkStrategy.for_monotonous_timestamps(), + source_name='kafka source') + ds.print() + plan = json.loads(self.env.get_execution_plan()) + self.assertEqual('Source: kafka source', plan['nodes'][0]['type']) + + def test_set_properties(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_group_id('test_group_id') \ + .set_client_id_prefix('test_client_id_prefix') \ + .set_property('test_property', 'test_value') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + conf = self._get_kafka_source_configuration(source) + self.assertEqual(conf.get_string('bootstrap.servers', ''), 'localhost:9092') + self.assertEqual(conf.get_string('group.id', ''), 'test_group_id') + self.assertEqual(conf.get_string('client.id.prefix', ''), 'test_client_id_prefix') + self.assertEqual(conf.get_string('test_property', ''), 'test_value') + + def test_set_topics(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic1', 'test_topic2') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicListSubscriber' + ) + topics = get_field_value(kafka_subscriber, 'topics') + self.assertTrue(is_instance_of(topics, get_gateway().jvm.java.util.List)) + self.assertEqual(topics.size(), 2) + self.assertEqual(topics[0], 'test_topic1') + self.assertEqual(topics[1], 'test_topic2') + + def test_set_topic_pattern(self): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topic_pattern('test_topic*') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.TopicPatternSubscriber' + ) + topic_pattern = get_field_value(kafka_subscriber, 'topicPattern') + self.assertTrue(is_instance_of(topic_pattern, get_gateway().jvm.java.util.regex.Pattern)) + self.assertEqual(topic_pattern.toString(), 'test_topic*') + + def test_set_partitions(self): + topic_partition_1 = KafkaTopicPartition('test_topic', 1) + topic_partition_2 = KafkaTopicPartition('test_topic', 2) + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_partitions({topic_partition_1, topic_partition_2}) \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .build() + kafka_subscriber = get_field_value(source.get_java_function(), 'subscriber') + self.assertEqual( + kafka_subscriber.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.subscriber.PartitionSetSubscriber' + ) + partitions = get_field_value(kafka_subscriber, 'subscribedPartitions') + self.assertTrue(is_instance_of(partitions, get_gateway().jvm.java.util.Set)) + self.assertTrue(topic_partition_1._to_j_topic_partition() in partitions) + self.assertTrue(topic_partition_2._to_j_topic_partition() in partitions) + + def test_set_starting_offsets(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_starting_offsets(initializer) \ + .build() + + self._check_latest_offsets_initializer( + _build_source(KafkaOffsetsInitializer.latest())) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.earliest()), -2, + KafkaOffsetResetStrategy.EARLIEST + ) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.committed_offsets()), -3, + KafkaOffsetResetStrategy.NONE + ) + self._check_reader_handled_offsets_initializer( + _build_source(KafkaOffsetsInitializer.committed_offsets( + KafkaOffsetResetStrategy.LATEST + )), -3, KafkaOffsetResetStrategy.LATEST + ) + self._check_timestamp_offsets_initializer( + _build_source(KafkaOffsetsInitializer.timestamp(100)), 100 + ) + specified_offsets = { + KafkaTopicPartition('test_topic1', 1): 1000, + KafkaTopicPartition('test_topic2', 2): 2000 + } + self._check_specified_offsets_initializer( + _build_source(KafkaOffsetsInitializer.offsets(specified_offsets)), specified_offsets, + KafkaOffsetResetStrategy.EARLIEST + ) + self._check_specified_offsets_initializer( + _build_source(KafkaOffsetsInitializer.offsets( + specified_offsets, + KafkaOffsetResetStrategy.LATEST + )), + specified_offsets, + KafkaOffsetResetStrategy.LATEST + ) + + def test_bounded(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_bounded(initializer) \ + .build() + + def _check_bounded(source: KafkaSource): + self.assertEqual( + get_field_value(source.get_java_function(), 'boundedness').toString(), 'BOUNDED' + ) + + self._test_set_bounded_or_unbounded(_build_source, _check_bounded) + + def test_unbounded(self): + def _build_source(initializer: KafkaOffsetsInitializer): + return KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(SimpleStringSchema()) \ + .set_group_id('test_group') \ + .set_unbounded(initializer) \ + .build() + + def _check_bounded(source: KafkaSource): + self.assertEqual( + get_field_value(source.get_java_function(), 'boundedness').toString(), + 'CONTINUOUS_UNBOUNDED' + ) + + self._test_set_bounded_or_unbounded(_build_source, _check_bounded) + + def _test_set_bounded_or_unbounded(self, _build_source, _check_boundedness): + source = _build_source(KafkaOffsetsInitializer.latest()) + _check_boundedness(source) + self._check_latest_offsets_initializer(source, False) + source = _build_source(KafkaOffsetsInitializer.earliest()) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -2, KafkaOffsetResetStrategy.EARLIEST, False + ) + source = _build_source(KafkaOffsetsInitializer.committed_offsets()) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -3, KafkaOffsetResetStrategy.NONE, False + ) + source = _build_source(KafkaOffsetsInitializer.committed_offsets( + KafkaOffsetResetStrategy.LATEST + )) + _check_boundedness(source) + self._check_reader_handled_offsets_initializer( + source, -3, KafkaOffsetResetStrategy.LATEST, False + ) + source = _build_source(KafkaOffsetsInitializer.timestamp(100)) + _check_boundedness(source) + self._check_timestamp_offsets_initializer(source, 100, False) + specified_offsets = { + KafkaTopicPartition('test_topic1', 1): 1000, + KafkaTopicPartition('test_topic2', 2): 2000 + } + source = _build_source(KafkaOffsetsInitializer.offsets(specified_offsets)) + _check_boundedness(source) + self._check_specified_offsets_initializer( + source, specified_offsets, KafkaOffsetResetStrategy.EARLIEST, False + ) + source = _build_source(KafkaOffsetsInitializer.offsets( + specified_offsets, + KafkaOffsetResetStrategy.LATEST) + ) + _check_boundedness(source) + self._check_specified_offsets_initializer( + source, specified_offsets, KafkaOffsetResetStrategy.LATEST, False + ) + + def test_set_value_only_deserializer(self): + def _check(schema: DeserializationSchema, class_name: str): + source = KafkaSource.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_topics('test_topic') \ + .set_value_only_deserializer(schema) \ + .build() + deserialization_schema_wrapper = get_field_value(source.get_java_function(), + 'deserializationSchema') + self.assertEqual( + deserialization_schema_wrapper.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.reader.deserializer' + '.KafkaValueOnlyDeserializationSchemaWrapper' + ) + deserialization_schema = get_field_value(deserialization_schema_wrapper, + 'deserializationSchema') + self.assertEqual(deserialization_schema.getClass().getCanonicalName(), + class_name) + + _check(SimpleStringSchema(), 'org.apache.flink.api.common.serialization.SimpleStringSchema') + _check( + JsonRowDeserializationSchema.builder().type_info(Types.ROW([Types.STRING()])).build(), + 'org.apache.flink.formats.json.JsonRowDeserializationSchema' + ) + _check( + CsvRowDeserializationSchema.Builder(Types.ROW([Types.STRING()])).build(), + 'org.apache.flink.formats.csv.CsvRowDeserializationSchema' + ) + avro_schema_string = """ + { + "type": "record", + "name": "test_record", + "fields": [] + } + """ + _check( + AvroRowDeserializationSchema(avro_schema_string=avro_schema_string), + 'org.apache.flink.formats.avro.AvroRowDeserializationSchema' + ) + + def _check_reader_handled_offsets_initializer(self, + source: KafkaSource, + offset: int, + reset_strategy: KafkaOffsetResetStrategy, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.ReaderHandledOffsetsInitializer' + ) + + starting_offset = get_field_value(offsets_initializer, 'startingOffset') + self.assertEqual(starting_offset, offset) + + offset_reset_strategy = get_field_value(offsets_initializer, 'offsetResetStrategy') + self.assertTrue( + offset_reset_strategy.equals(reset_strategy._to_j_offset_reset_strategy()) + ) + + def _check_latest_offsets_initializer(self, + source: KafkaSource, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.LatestOffsetsInitializer' + ) + + def _check_timestamp_offsets_initializer(self, + source: KafkaSource, + timestamp: int, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.TimestampOffsetsInitializer' + ) + + starting_timestamp = get_field_value(offsets_initializer, 'startingTimestamp') + self.assertEqual(starting_timestamp, timestamp) + + def _check_specified_offsets_initializer(self, + source: KafkaSource, + offsets: Dict[KafkaTopicPartition, int], + reset_strategy: KafkaOffsetResetStrategy, + is_start: bool = True): + if is_start: + field_name = 'startingOffsetsInitializer' + else: + field_name = 'stoppingOffsetsInitializer' + offsets_initializer = get_field_value(source.get_java_function(), field_name) + self.assertEqual( + offsets_initializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.source.enumerator.initializer' + '.SpecifiedOffsetsInitializer' + ) + + initial_offsets = get_field_value(offsets_initializer, 'initialOffsets') + self.assertTrue(is_instance_of(initial_offsets, get_gateway().jvm.java.util.Map)) + self.assertEqual(initial_offsets.size(), len(offsets)) + for j_topic_partition in initial_offsets: + topic_partition = KafkaTopicPartition(j_topic_partition.topic(), + j_topic_partition.partition()) + self.assertIsNotNone(offsets.get(topic_partition)) + self.assertEqual(initial_offsets[j_topic_partition], offsets[topic_partition]) + + offset_reset_strategy = get_field_value(offsets_initializer, 'offsetResetStrategy') + self.assertTrue( + offset_reset_strategy.equals(reset_strategy._to_j_offset_reset_strategy()) + ) + + @staticmethod + def _get_kafka_source_configuration(source: KafkaSource): + jvm = get_gateway().jvm + j_source = source.get_java_function() + j_to_configuration = j_source.getClass().getDeclaredMethod( + 'getConfiguration', to_jarray(jvm.java.lang.Class, []) + ) + j_to_configuration.setAccessible(True) + j_configuration = j_to_configuration.invoke(j_source, to_jarray(jvm.java.lang.Object, [])) + return Configuration(j_configuration=j_configuration) + + +class KafkaSinkTests(PyFlinkStreamingTestCase): + + def test_compile(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + + ds = self.env.from_collection([], type_info=Types.STRING()) + ds.sink_to(sink) + + plan = json.loads(self.env.get_execution_plan()) + self.assertEqual(plan['nodes'][1]['type'], 'Sink: Writer') + self.assertEqual(plan['nodes'][2]['type'], 'Sink: Committer') + + def test_set_bootstrap_severs(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092,localhost:9093') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + config = get_field_value(sink.get_java_function(), 'kafkaProducerConfig') + self.assertEqual(config.get('bootstrap.servers'), 'localhost:9092,localhost:9093') + + def test_set_delivery_guarantee(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'none') + + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_delivery_guarantee(DeliveryGuarantee.AT_LEAST_ONCE) \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'at-least-once') + + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_delivery_guarantee(DeliveryGuarantee.EXACTLY_ONCE) \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + guarantee = get_field_value(sink.get_java_function(), 'deliveryGuarantee') + self.assertEqual(guarantee.toString(), 'exactly-once') + + def test_set_transactional_id_prefix(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_transactional_id_prefix('test-prefix') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + prefix = get_field_value(sink.get_java_function(), 'transactionalIdPrefix') + self.assertEqual(prefix, 'test-prefix') + + def test_set_property(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .set_property('test-key', 'test-value') \ + .build() + config = get_field_value(sink.get_java_function(), 'kafkaProducerConfig') + self.assertEqual(config.get('test-key'), 'test-value') + + def test_set_record_serializer(self): + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(self._build_serialization_schema()) \ + .build() + serializer = get_field_value(sink.get_java_function(), 'recordSerializer') + self.assertEqual(serializer.getClass().getCanonicalName(), + 'org.apache.flink.connector.kafka.sink.' + 'KafkaRecordSerializationSchemaBuilder.' + 'KafkaRecordSerializationSchemaWrapper') + topic_selector = get_field_value(serializer, 'topicSelector') + self.assertEqual(topic_selector.apply(None), 'test-topic') + value_serializer = get_field_value(serializer, 'valueSerializationSchema') + self.assertEqual(value_serializer.getClass().getCanonicalName(), + 'org.apache.flink.api.common.serialization.SimpleStringSchema') + + @staticmethod + def _build_serialization_schema() -> KafkaRecordSerializationSchema: + return KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema(SimpleStringSchema()) \ + .build() + + +class KafkaRecordSerializationSchemaTests(PyFlinkTestCase): + + def test_set_topic(self): + input_type = Types.ROW([Types.STRING()]) + + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema( + JsonRowSerializationSchema.builder().with_type_info(input_type).build()) \ + .build() + jvm = get_gateway().jvm + serialization_schema._j_serialization_schema.open( + jvm.org.apache.flink.connector.testutils.formats.DummyInitializationContext(), + jvm.org.apache.flink.connector.kafka.sink.DefaultKafkaSinkContext( + 0, 1, jvm.java.util.Properties())) + + j_record = serialization_schema._j_serialization_schema.serialize( + to_java_data_structure(Row('test')), None, None + ) + self.assertEqual(j_record.topic(), 'test-topic') + self.assertIsNone(j_record.key()) + self.assertEqual(j_record.value(), b'{"f0":"test"}') + + def test_set_topic_selector(self): + def _select(data): + data = data[0] + if data == 'a': + return 'topic-a' + elif data == 'b': + return 'topic-b' + else: + return 'topic-dead-letter' + + def _check_record(data, topic, serialized_data): + input_type = Types.ROW([Types.STRING()]) + + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic_selector(_select) \ + .set_value_serialization_schema( + JsonRowSerializationSchema.builder().with_type_info(input_type).build()) \ + .build() + jvm = get_gateway().jvm + serialization_schema._j_serialization_schema.open( + jvm.org.apache.flink.connector.testutils.formats.DummyInitializationContext(), + jvm.org.apache.flink.connector.kafka.sink.DefaultKafkaSinkContext( + 0, 1, jvm.java.util.Properties())) + sink = KafkaSink.builder() \ + .set_bootstrap_servers('localhost:9092') \ + .set_record_serializer(serialization_schema) \ + .build() + + ds = MockDataStream(Types.ROW([Types.STRING()])) + ds.sink_to(sink) + row = Row(data) + topic_row = ds.feed(row) # type: Row + j_record = serialization_schema._j_serialization_schema.serialize( + to_java_data_structure(topic_row), None, None + ) + self.assertEqual(j_record.topic(), topic) + self.assertIsNone(j_record.key()) + self.assertEqual(j_record.value(), serialized_data) + + _check_record('a', 'topic-a', b'{"f0":"a"}') + _check_record('b', 'topic-b', b'{"f0":"b"}') + _check_record('c', 'topic-dead-letter', b'{"f0":"c"}') + _check_record('d', 'topic-dead-letter', b'{"f0":"d"}') + + def test_set_key_serialization_schema(self): + def _check_key_serialization_schema(key_serialization_schema, expected_class): + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_key_serialization_schema(key_serialization_schema) \ + .set_value_serialization_schema(SimpleStringSchema()) \ + .build() + schema_field = get_field_value(serialization_schema._j_serialization_schema, + 'keySerializationSchema') + self.assertIsNotNone(schema_field) + self.assertEqual(schema_field.getClass().getCanonicalName(), expected_class) + + self._check_serialization_schema_implementations(_check_key_serialization_schema) + + def test_set_value_serialization_schema(self): + def _check_value_serialization_schema(value_serialization_schema, expected_class): + serialization_schema = KafkaRecordSerializationSchema.builder() \ + .set_topic('test-topic') \ + .set_value_serialization_schema(value_serialization_schema) \ + .build() + schema_field = get_field_value(serialization_schema._j_serialization_schema, + 'valueSerializationSchema') + self.assertIsNotNone(schema_field) + self.assertEqual(schema_field.getClass().getCanonicalName(), expected_class) + + self._check_serialization_schema_implementations(_check_value_serialization_schema) + + @staticmethod + def _check_serialization_schema_implementations(check_function): + input_type = Types.ROW([Types.STRING()]) + + check_function( + JsonRowSerializationSchema.builder().with_type_info(input_type).build(), + 'org.apache.flink.formats.json.JsonRowSerializationSchema' + ) + check_function( + CsvRowSerializationSchema.Builder(input_type).build(), + 'org.apache.flink.formats.csv.CsvRowSerializationSchema' + ) + avro_schema_string = """ + { + "type": "record", + "name": "test_record", + "fields": [] + } + """ + check_function( + AvroRowSerializationSchema(avro_schema_string=avro_schema_string), + 'org.apache.flink.formats.avro.AvroRowSerializationSchema' + ) + check_function( + SimpleStringSchema(), + 'org.apache.flink.api.common.serialization.SimpleStringSchema' + ) + + +class MockDataStream(data_stream.DataStream): + + def __init__(self, original_type=None): + super().__init__(None) + self._operators = [] + self._type = original_type + + def feed(self, data): + for op in self._operators: + data = op(data) + return data + + def get_type(self): + return self._type + + def map(self, f, output_type=None): + self._operators.append(f) + self._type = output_type + + def sink_to(self, sink): + ds = self + from pyflink.datastream.connectors.base import SupportsPreprocessing + if isinstance(sink, SupportsPreprocessing) and sink.get_transformer() is not None: + ds = sink.get_transformer().apply(self) + return ds diff --git a/flink-python/pyflink/flink-conf.yaml b/flink-python/pyflink/flink-conf.yaml new file mode 100644 index 000000000..b5aa2794d --- /dev/null +++ b/flink-python/pyflink/flink-conf.yaml @@ -0,0 +1,311 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ + +# These parameters are required for Java 17 support. +# They can be safely removed when using Java 8/11. +env.java.opts.all: --add-exports=java.base/sun.net.util=ALL-UNNAMED --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.text=ALL-UNNAMED --add-opens=java.base/java.time=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.locks=ALL-UNNAMED + +#============================================================================== +# Common +#============================================================================== + +# The external address of the host on which the JobManager runs and can be +# reached by the TaskManagers and any clients which want to connect. This setting +# is only used in Standalone mode and may be overwritten on the JobManager side +# by specifying the --host parameter of the bin/jobmanager.sh executable. +# In high availability mode, if you use the bin/start-cluster.sh script and setup +# the conf/masters file, this will be taken care of automatically. Yarn +# automatically configure the host name based on the hostname of the node where the +# JobManager runs. + +jobmanager.rpc.address: localhost + +# The RPC port where the JobManager is reachable. + +jobmanager.rpc.port: 6123 + +# The host interface the JobManager will bind to. By default, this is localhost, and will prevent +# the JobManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. +# +# To enable this, set the bind-host address to one that has access to an outside facing network +# interface, such as 0.0.0.0. + +jobmanager.bind-host: localhost + + +# The total process memory size for the JobManager. +# +# Note this accounts for all memory usage within the JobManager process, including JVM metaspace and other overhead. + +jobmanager.memory.process.size: 1600m + +# The host interface the TaskManager will bind to. By default, this is localhost, and will prevent +# the TaskManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. +# +# To enable this, set the bind-host address to one that has access to an outside facing network +# interface, such as 0.0.0.0. + +taskmanager.bind-host: localhost + +# The address of the host on which the TaskManager runs and can be reached by the JobManager and +# other TaskManagers. If not specified, the TaskManager will try different strategies to identify +# the address. +# +# Note this address needs to be reachable by the JobManager and forward traffic to one of +# the interfaces the TaskManager is bound to (see 'taskmanager.bind-host'). +# +# Note also that unless all TaskManagers are running on the same machine, this address needs to be +# configured separately for each TaskManager. + +taskmanager.host: localhost + +# The total process memory size for the TaskManager. +# +# Note this accounts for all memory usage within the TaskManager process, including JVM metaspace and other overhead. + +taskmanager.memory.process.size: 1728m + +# To exclude JVM metaspace and overhead, please, use total Flink memory size instead of 'taskmanager.memory.process.size'. +# It is not recommended to set both 'taskmanager.memory.process.size' and Flink memory. +# +# taskmanager.memory.flink.size: 1280m + +# The number of task slots that each TaskManager offers. Each slot runs one parallel pipeline. + +taskmanager.numberOfTaskSlots: 1 + +# The parallelism used for programs that did not specify and other parallelism. + +parallelism.default: 1 + +# The default file system scheme and authority. +# +# By default file paths without scheme are interpreted relative to the local +# root file system 'file:///'. Use this to override the default and interpret +# relative paths relative to a different file system, +# for example 'hdfs://mynamenode:12345' +# +# fs.default-scheme + +#============================================================================== +# High Availability +#============================================================================== + +# The high-availability mode. Possible options are 'NONE' or 'zookeeper'. +# +# high-availability.type: zookeeper + +# The path where metadata for master recovery is persisted. While ZooKeeper stores +# the small ground truth for checkpoint and leader election, this location stores +# the larger objects, like persisted dataflow graphs. +# +# Must be a durable file system that is accessible from all nodes +# (like HDFS, S3, Ceph, nfs, ...) +# +# high-availability.storageDir: hdfs:///flink/ha/ + +# The list of ZooKeeper quorum peers that coordinate the high-availability +# setup. This must be a list of the form: +# "host1:clientPort,host2:clientPort,..." (default clientPort: 2181) +# +# high-availability.zookeeper.quorum: localhost:2181 + + +# ACL options are based on https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_BuiltinACLSchemes +# It can be either "creator" (ZOO_CREATE_ALL_ACL) or "open" (ZOO_OPEN_ACL_UNSAFE) +# The default value is "open" and it can be changed to "creator" if ZK security is enabled +# +# high-availability.zookeeper.client.acl: open + +#============================================================================== +# Fault tolerance and checkpointing +#============================================================================== + +# The backend that will be used to store operator state checkpoints if +# checkpointing is enabled. Checkpointing is enabled when execution.checkpointing.interval > 0. +# +# Execution checkpointing related parameters. Please refer to CheckpointConfig and ExecutionCheckpointingOptions for more details. +# +# execution.checkpointing.interval: 3min +# execution.checkpointing.externalized-checkpoint-retention: [DELETE_ON_CANCELLATION, RETAIN_ON_CANCELLATION] +# execution.checkpointing.max-concurrent-checkpoints: 1 +# execution.checkpointing.min-pause: 0 +# execution.checkpointing.mode: [EXACTLY_ONCE, AT_LEAST_ONCE] +# execution.checkpointing.timeout: 10min +# execution.checkpointing.tolerable-failed-checkpoints: 0 +# execution.checkpointing.unaligned: false +# +# Supported backends are 'hashmap', 'rocksdb', or the +# . +# +# state.backend.type: hashmap + +# Directory for checkpoints filesystem, when using any of the default bundled +# state backends. +# +# state.checkpoints.dir: hdfs://namenode-host:port/flink-checkpoints + +# Default target directory for savepoints, optional. +# +# state.savepoints.dir: hdfs://namenode-host:port/flink-savepoints + +# Flag to enable/disable incremental checkpoints for backends that +# support incremental checkpoints (like the RocksDB state backend). +# +# state.backend.incremental: false + +# The failover strategy, i.e., how the job computation recovers from task failures. +# Only restart tasks that may have been affected by the task failure, which typically includes +# downstream tasks and potentially upstream tasks if their produced data is no longer available for consumption. + +jobmanager.execution.failover-strategy: region + +#============================================================================== +# Rest & web frontend +#============================================================================== + +# The port to which the REST client connects to. If rest.bind-port has +# not been specified, then the server will bind to this port as well. +# +#rest.port: 8081 + +# The address to which the REST client will connect to +# +rest.address: localhost + +# Port range for the REST and web server to bind to. +# +#rest.bind-port: 8080-8090 + +# The address that the REST & web server binds to +# By default, this is localhost, which prevents the REST & web server from +# being able to communicate outside of the machine/container it is running on. +# +# To enable this, set the bind address to one that has access to outside-facing +# network interface, such as 0.0.0.0. +# +rest.bind-address: localhost + +# Flag to specify whether job submission is enabled from the web-based +# runtime monitor. Uncomment to disable. + +#web.submit.enable: false + +# Flag to specify whether job cancellation is enabled from the web-based +# runtime monitor. Uncomment to disable. + +#web.cancel.enable: false + +#============================================================================== +# Advanced +#============================================================================== + +# Override the directories for temporary files. If not specified, the +# system-specific Java temporary directory (java.io.tmpdir property) is taken. +# +# For framework setups on Yarn, Flink will automatically pick up the +# containers' temp directories without any need for configuration. +# +# Add a delimited list for multiple directories, using the system directory +# delimiter (colon ':' on unix) or a comma, e.g.: +# /data1/tmp:/data2/tmp:/data3/tmp +# +# Note: Each directory entry is read from and written to by a different I/O +# thread. You can include the same directory multiple times in order to create +# multiple I/O threads against that directory. This is for example relevant for +# high-throughput RAIDs. +# +# io.tmp.dirs: /tmp + +# The classloading resolve order. Possible values are 'child-first' (Flink's default) +# and 'parent-first' (Java's default). +# +# Child first classloading allows users to use different dependency/library +# versions in their application than those in the classpath. Switching back +# to 'parent-first' may help with debugging dependency issues. +# +# classloader.resolve-order: child-first + +# The amount of memory going to the network stack. These numbers usually need +# no tuning. Adjusting them may be necessary in case of an "Insufficient number +# of network buffers" error. The default min is 64MB, the default max is 1GB. +# +# taskmanager.memory.network.fraction: 0.1 +# taskmanager.memory.network.min: 64mb +# taskmanager.memory.network.max: 1gb + +#============================================================================== +# Flink Cluster Security Configuration +#============================================================================== + +# Kerberos authentication for various components - Hadoop, ZooKeeper, and connectors - +# may be enabled in four steps: +# 1. configure the local krb5.conf file +# 2. provide Kerberos credentials (either a keytab or a ticket cache w/ kinit) +# 3. make the credentials available to various JAAS login contexts +# 4. configure the connector to use JAAS/SASL + +# The below configure how Kerberos credentials are provided. A keytab will be used instead of +# a ticket cache if the keytab path and principal are set. + +# security.kerberos.login.use-ticket-cache: true +# security.kerberos.login.keytab: /path/to/kerberos/keytab +# security.kerberos.login.principal: flink-user + +# The configuration below defines which JAAS login contexts + +# security.kerberos.login.contexts: Client,KafkaClient + +#============================================================================== +# ZK Security Configuration +#============================================================================== + +# Below configurations are applicable if ZK ensemble is configured for security + +# Override below configuration to provide custom ZK service name if configured +# zookeeper.sasl.service-name: zookeeper + +# The configuration below must match one of the values set in "security.kerberos.login.contexts" +# zookeeper.sasl.login-context-name: Client + +#============================================================================== +# HistoryServer +#============================================================================== + +# The HistoryServer is started and stopped via bin/historyserver.sh (start|stop) + +# Directory to upload completed jobs to. Add this directory to the list of +# monitored directories of the HistoryServer as well (see below). +#jobmanager.archive.fs.dir: hdfs:///completed-jobs/ + +# The address under which the web-based HistoryServer listens. +#historyserver.web.address: 0.0.0.0 + +# The port under which the web-based HistoryServer listens. +#historyserver.web.port: 8082 + +# Comma separated list of directories to monitor for completed jobs. +#historyserver.archive.fs.dir: hdfs:///completed-jobs/ + +# Interval in milliseconds for refreshing the monitored directories. +#historyserver.archive.fs.refresh-interval: 10000 + diff --git a/flink-python/pyflink/pyflink_gateway_server.py b/flink-python/pyflink/pyflink_gateway_server.py new file mode 100644 index 000000000..1cf25a54f --- /dev/null +++ b/flink-python/pyflink/pyflink_gateway_server.py @@ -0,0 +1,288 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ + +# This is a copy of the pyflink_gateway_server.py file from the Flink. +# The original file which is accessible here: +# https://github.com/apache/flink/blob/master/flink-python/pyflink/pyflink_gateway_server.py +# Additional change is the handling of the FLINK_TEST_LIB_DIR environmental variable. +# It could be used to add extra testing jars for the gateway classpath. +# The plan is to remove this once Pyflink 1.19 is released + +import argparse +import getpass +import glob +import os +import platform +import signal +import socket +import sys +from collections import namedtuple +from string import Template +from subprocess import Popen, PIPE + +from pyflink.find_flink_home import _find_flink_home, _find_flink_source_root + +KEY_ENV_LOG_DIR = "env.log.dir" +KEY_ENV_YARN_CONF_DIR = "env.yarn.conf.dir" +KEY_ENV_HADOOP_CONF_DIR = "env.hadoop.conf.dir" +KEY_ENV_HBASE_CONF_DIR = "env.hbase.conf.dir" +KEY_ENV_JAVA_HOME = "env.java.home" +KEY_ENV_JAVA_OPTS = "env.java.opts.all" +KEY_ENV_JAVA_OPTS_DEPRECATED = "env.java.opts" + + +def on_windows(): + return platform.system() == "Windows" + + +def read_from_config(key, default_value, flink_conf_file): + value = default_value + # get the realpath of tainted path value to avoid CWE22 problem that constructs a path or URI + # using the tainted value and might allow an attacker to access, modify, or test the existence + # of critical or sensitive files. + with open(os.path.realpath(flink_conf_file), "r") as f: + while True: + line = f.readline() + if not line: + break + if line.startswith("#") or len(line.strip()) == 0: + continue + k, v = line.split(":", 1) + if k.strip() == key: + value = v.strip() + return value + + +def find_java_executable(): + java_executable = "java.exe" if on_windows() else "java" + flink_home = _find_flink_home() + flink_conf_file = os.path.join(flink_home, "conf", "flink-conf.yaml") + java_home = read_from_config(KEY_ENV_JAVA_HOME, None, flink_conf_file) + + if java_home is None and "JAVA_HOME" in os.environ: + java_home = os.environ["JAVA_HOME"] + + if java_home is not None: + java_executable = os.path.join(java_home, "bin", java_executable) + + return java_executable + + +def prepare_environment_variables(env): + flink_home = _find_flink_home() + # get the realpath of tainted path value to avoid CWE22 problem that constructs a path or URI + # using the tainted value and might allow an attacker to access, modify, or test the existence + # of critical or sensitive files. + real_flink_home = os.path.realpath(flink_home) + + if 'FLINK_CONF_DIR' in env: + flink_conf_directory = os.path.realpath(env['FLINK_CONF_DIR']) + else: + flink_conf_directory = os.path.join(real_flink_home, "conf") + env['FLINK_CONF_DIR'] = flink_conf_directory + + if 'FLINK_LIB_DIR' in env: + flink_lib_directory = os.path.realpath(env['FLINK_LIB_DIR']) + else: + flink_lib_directory = os.path.join(real_flink_home, "lib") + env['FLINK_LIB_DIR'] = flink_lib_directory + + if 'FLINK_OPT_DIR' in env: + flink_opt_directory = os.path.realpath(env['FLINK_OPT_DIR']) + else: + flink_opt_directory = os.path.join(real_flink_home, "opt") + env['FLINK_OPT_DIR'] = flink_opt_directory + + if 'FLINK_PLUGINS_DIR' in env: + flink_plugins_directory = os.path.realpath(env['FLINK_PLUGINS_DIR']) + else: + flink_plugins_directory = os.path.join(real_flink_home, "plugins") + env['FLINK_PLUGINS_DIR'] = flink_plugins_directory + + env["FLINK_BIN_DIR"] = os.path.join(real_flink_home, "bin") + + +def construct_log_settings(env): + templates = [ + "-Dlog.file=${flink_log_dir}/flink-${flink_ident_string}-python-${hostname}.log", + "-Dlog4j.configuration=${log4j_properties}", + "-Dlog4j.configurationFile=${log4j_properties}", + "-Dlogback.configurationFile=${logback_xml}" + ] + + flink_home = os.path.realpath(_find_flink_home()) + flink_conf_dir = env['FLINK_CONF_DIR'] + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + + if "FLINK_LOG_DIR" in env: + flink_log_dir = env["FLINK_LOG_DIR"] + else: + flink_log_dir = read_from_config( + KEY_ENV_LOG_DIR, os.path.join(flink_home, "log"), flink_conf_file) + + if "LOG4J_PROPERTIES" in env: + log4j_properties = env["LOG4J_PROPERTIES"] + else: + log4j_properties = "%s/log4j-cli.properties" % flink_conf_dir + + if "LOGBACK_XML" in env: + logback_xml = env["LOGBACK_XML"] + else: + logback_xml = "%s/logback.xml" % flink_conf_dir + + if "FLINK_IDENT_STRING" in env: + flink_ident_string = env["FLINK_IDENT_STRING"] + else: + flink_ident_string = getpass.getuser() + + hostname = socket.gethostname() + log_settings = [] + for template in templates: + log_settings.append(Template(template).substitute( + log4j_properties=log4j_properties, + logback_xml=logback_xml, + flink_log_dir=flink_log_dir, + flink_ident_string=flink_ident_string, + hostname=hostname)) + return log_settings + + +def get_jvm_opts(env): + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + jvm_opts = env.get( + 'FLINK_ENV_JAVA_OPTS', + read_from_config( + KEY_ENV_JAVA_OPTS, + read_from_config(KEY_ENV_JAVA_OPTS_DEPRECATED, "", flink_conf_file), + flink_conf_file)) + + # Remove leading and ending double quotes (if present) of value + jvm_opts = jvm_opts.strip("\"") + return jvm_opts.split(" ") + + +def construct_flink_classpath(env): + flink_home = _find_flink_home() + flink_lib_directory = env['FLINK_LIB_DIR'] + flink_opt_directory = env['FLINK_OPT_DIR'] + + if on_windows(): + # The command length is limited on Windows. To avoid the problem we should shorten the + # command length as much as possible. + lib_jars = os.path.join(flink_lib_directory, "*") + else: + lib_jars = os.pathsep.join(glob.glob(os.path.join(flink_lib_directory, "*.jar"))) + + flink_python_jars = glob.glob(os.path.join(flink_opt_directory, "flink-python*.jar")) + if len(flink_python_jars) < 1: + print("The flink-python jar is not found in the opt folder of the FLINK_HOME: %s" % + flink_home) + return lib_jars + flink_python_jar = flink_python_jars[0] + + return os.pathsep.join([lib_jars, flink_python_jar]) + + +def construct_hadoop_classpath(env): + flink_conf_file = os.path.join(env['FLINK_CONF_DIR'], "flink-conf.yaml") + + hadoop_conf_dir = "" + if 'HADOOP_CONF_DIR' not in env and 'HADOOP_CLASSPATH' not in env: + if os.path.isdir("/etc/hadoop/conf"): + print("Setting HADOOP_CONF_DIR=/etc/hadoop/conf because no HADOOP_CONF_DIR or" + "HADOOP_CLASSPATH was set.") + hadoop_conf_dir = "/etc/hadoop/conf" + + hbase_conf_dir = "" + if 'HBASE_CONF_DIR' not in env: + if os.path.isdir("/etc/hbase/conf"): + print("Setting HBASE_CONF_DIR=/etc/hbase/conf because no HBASE_CONF_DIR was set.") + hbase_conf_dir = "/etc/hbase/conf" + + return os.pathsep.join( + [env.get("HADOOP_CLASSPATH", ""), + env.get("YARN_CONF_DIR", + read_from_config(KEY_ENV_YARN_CONF_DIR, "", flink_conf_file)), + env.get("HADOOP_CONF_DIR", + read_from_config(KEY_ENV_HADOOP_CONF_DIR, hadoop_conf_dir, flink_conf_file)), + env.get("HBASE_CONF_DIR", + read_from_config(KEY_ENV_HBASE_CONF_DIR, hbase_conf_dir, flink_conf_file))]) + + +def construct_test_classpath(env): + test_jar_patterns = [ + "flink-python/target/test-dependencies/*", + "flink-python/target/artifacts/testDataStream.jar", + "flink-python/target/flink-python*-tests.jar", + ] + test_jars = [] + + # Connector tests need to add specific jars to the gateway classpath + if 'FLINK_TEST_LIBS' in env: + test_jars += glob.glob(env['FLINK_TEST_LIBS']) + else: + flink_source_root = _find_flink_source_root() + for pattern in test_jar_patterns: + pattern = pattern.replace("/", os.path.sep) + test_jars += glob.glob(os.path.join(flink_source_root, pattern)) + return os.path.pathsep.join(test_jars) + + +def construct_program_args(args): + parser = argparse.ArgumentParser() + parser.add_argument("-c", "--class", required=True) + parser.add_argument("cluster_type", choices=["local", "remote", "yarn"]) + parse_result, other_args = parser.parse_known_args(args) + main_class = getattr(parse_result, "class") + cluster_type = parse_result.cluster_type + return namedtuple( + "ProgramArgs", ["main_class", "cluster_type", "other_args"])( + main_class, cluster_type, other_args) + + +def launch_gateway_server_process(env, args): + prepare_environment_variables(env) + program_args = construct_program_args(args) + if program_args.cluster_type == "local": + java_executable = find_java_executable() + log_settings = construct_log_settings(env) + jvm_args = env.get('JVM_ARGS', '') + jvm_opts = get_jvm_opts(env) + classpath = os.pathsep.join( + [construct_flink_classpath(env), construct_hadoop_classpath(env)]) + if "FLINK_TESTING" in env: + classpath = os.pathsep.join([classpath, construct_test_classpath(env)]) + command = [java_executable, jvm_args, "-XX:+IgnoreUnrecognizedVMOptions", + "--add-opens=jdk.proxy2/jdk.proxy2=ALL-UNNAMED"] \ + + jvm_opts + log_settings \ + + ["-cp", classpath, program_args.main_class] + program_args.other_args + else: + command = [os.path.join(env["FLINK_BIN_DIR"], "flink"), "run"] + program_args.other_args \ + + ["-c", program_args.main_class] + preexec_fn = None + if not on_windows(): + def preexec_func(): + # ignore ctrl-c / SIGINT + signal.signal(signal.SIGINT, signal.SIG_IGN) + preexec_fn = preexec_func + return Popen(list(filter(lambda c: len(c) != 0, command)), + stdin=PIPE, stderr=PIPE, preexec_fn=preexec_fn, env=env) + + +if __name__ == "__main__": + launch_gateway_server_process(os.environ, sys.argv[1:]) diff --git a/flink-python/setup.py b/flink-python/setup.py new file mode 100644 index 000000000..8e788d4e5 --- /dev/null +++ b/flink-python/setup.py @@ -0,0 +1,158 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ +from __future__ import print_function + +import glob +import io +import os +import sys + +from setuptools import setup +from shutil import copy, rmtree +from xml.etree import ElementTree as ET + +PACKAGE_NAME = 'apache-flink-connector-kafka' +# Source files, directories +CURRENT_DIR = os.path.abspath(os.path.dirname(__file__)) +POM_FILE = os.path.join(CURRENT_DIR, '../pom.xml') +README_FILE = os.path.join(CURRENT_DIR, 'README.txt') + +# Generated files and directories +VERSION_FILE = os.path.join(CURRENT_DIR, 'pyflink/datastream/connectors/kafka_connector_version.py') +LIB_PATH = os.path.join(CURRENT_DIR, 'pyflink/lib') +DEPENDENCY_FILE = os.path.join(CURRENT_DIR, 'dev/dev-requirements.txt') + + +# Removes a file or directory if exists. +def remove_if_exists(file_path): + if os.path.exists(file_path): + if os.path.isfile(file_path): + os.remove(file_path) + if os.path.isdir(file_path): + rmtree(file_path) + + +# Reads the content of the README.txt file. +def readme_content(): + with io.open(README_FILE, 'r', encoding='utf-8') as f: + return f.read() + + +# Reads the parameters used by the setup command. +# The source is the kafka_connector_version.py and the README.txt. +def setup_parameters(): + try: + exec(open(VERSION_FILE).read()) + return locals()['__connector_version__'], locals()['__flink_dependency__'], readme_content() + except IOError: + print("Failed to load PyFlink version file for packaging. " + + "'%s' not found!" % VERSION_FILE, + file=sys.stderr) + sys.exit(-1) + + +# Reads and parses the flink-connector-kafka main pom.xml. +# Based on the version data in the pom.xml prepares the pyflink dir: +# - Generates kafka_connector_version.py +# - Generates dev-requirements.txt +# - Copies the flink-sql-connector-kafka*.jar to the pyflink/lib dir +def prepare_pyflink_dir(): + # source files + pom_root = ET.parse(POM_FILE).getroot() + flink_version = pom_root.findall( + "./{http://maven.apache.org/POM/4.0.0}properties/" + + "{http://maven.apache.org/POM/4.0.0}flink.version" + )[0].text + connector_version = pom_root.findall( + "./{http://maven.apache.org/POM/4.0.0}version")[0].text.replace("-SNAPSHOT", ".dev0") + + flink_dependency = "apache-flink>=" + flink_version + + os.makedirs(LIB_PATH) + connector_jar = \ + glob.glob(CURRENT_DIR + '/target/test-dependencies/flink-sql-connector-kafka*.jar')[0] + copy(connector_jar, LIB_PATH) + + with io.open(VERSION_FILE, 'w', encoding='utf-8') as f: + f.write('# Generated file, do not edit\n') + f.write('__connector_version__ = "' + connector_version + '"\n') + f.write('__flink_dependency__ = "' + flink_dependency + '"\n') + + with io.open(DEPENDENCY_FILE, 'w', encoding='utf-8') as f: + f.write('# Generated file, do not edit\n') + f.write(flink_dependency + '\n') + + +# Main +print("Python version used to package: " + sys.version) + +# Python version check +if sys.version_info < (3, 7): + print("Python versions prior to 3.7 are not supported for PyFlink.", + file=sys.stderr) + sys.exit(-1) + +# Checks the running environment: +# - In the connector source root directory - package preparation +# - Otherwise - package deployment +in_flink_source = os.path.isfile("../flink-connector-kafka/src/main/" + + "java/org/apache/flink/connector/kafka/source/KafkaSource.java") + +# Cleans up the generated files and directories and regenerate them. +if in_flink_source: + remove_if_exists(VERSION_FILE) + remove_if_exists(DEPENDENCY_FILE) + remove_if_exists(LIB_PATH) + prepare_pyflink_dir() + print("\nPreparing Flink Kafka connector package") + +# Reads the current setup data from the kafka_connector_version.py file and the README.txt +(connector_version, flink_dependency, long_description) = setup_parameters() + +print("\nConnector version: " + connector_version) +print("Flink dependency: " + flink_dependency + "\n") + +if in_flink_source: + # Removes temporary directory used by the setup tool + remove_if_exists(PACKAGE_NAME.replace('-', '_') + '.egg-info') + +# Runs the python setup +setup( + name=PACKAGE_NAME, + version=connector_version, + include_package_data=True, + url='https://flink.apache.org', + license='https://www.apache.org/licenses/LICENSE-2.0', + author='Apache Software Foundation', + author_email='dev@flink.apache.org', + python_requires='>=3.8', + install_requires=[flink_dependency], + description='Apache Flink Python Kafka Connector API', + long_description=long_description, + long_description_content_type='text/plain', + zip_safe=False, + classifiers=[ + 'Development Status :: 5 - Production/Stable', + 'License :: OSI Approved :: Apache Software License', + 'Programming Language :: Python :: 3.7', + 'Programming Language :: Python :: 3.8', + 'Programming Language :: Python :: 3.9', + 'Programming Language :: Python :: 3.10'] +) + +print("\nFlink Kafka connector package is ready\n") diff --git a/flink-python/tox.ini b/flink-python/tox.ini new file mode 100644 index 000000000..c21c00f7f --- /dev/null +++ b/flink-python/tox.ini @@ -0,0 +1,51 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +################################################################################ + +[tox] +# tox (https://tox.readthedocs.io/) is a tool for running tests +# in multiple virtualenvs. This configuration file will run the +# test suite on all supported python versions. +# new environments will be excluded by default unless explicitly added to envlist. +envlist = {py38, py39, py310}-cython + +[testenv] +whitelist_externals = /bin/bash +deps = apache-flink +passenv = * +commands = + python --version + pip install pytest + bash ./dev/integration_test.sh +# Replace the default installation command with a custom retry installation script, because on high-speed +# networks, downloading a package may raise a ConnectionResetError: [Errno 104] Peer reset connection. +install_command = {toxinidir}/dev/install_command.sh {opts} {packages} + +[flake8] +# We follow PEP 8 (https://www.python.org/dev/peps/pep-0008/) with one exception: lines can be +# up to 100 characters in length, not 79. +ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 +max-line-length=100 +exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/* + +[mypy] +files=pyflink/datastream/connectors/*.py +ignore_missing_imports = True +strict_optional=False + +[mypy-pyflink.fn_execution.*] +ignore_errors = True diff --git a/flink-sql-connector-kafka/pom.xml b/flink-sql-connector-kafka/pom.xml index 3be182c62..506e55d66 100644 --- a/flink-sql-connector-kafka/pom.xml +++ b/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 3.4-SNAPSHOT flink-sql-connector-kafka @@ -67,7 +67,6 @@ under the License. - org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kafka org.apache.kafka:* diff --git a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE index 641d4040d..926976da9 100644 --- a/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE +++ b/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka -Copyright 2014-2022 The Apache Software Foundation +Copyright 2014-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/mvnw b/mvnw new file mode 100755 index 000000000..684df0068 --- /dev/null +++ b/mvnw @@ -0,0 +1,259 @@ +#!/bin/sh +# ---------------------------------------------------------------------------- +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you 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. +# ---------------------------------------------------------------------------- + +# ---------------------------------------------------------------------------- +# Apache Maven Wrapper startup batch script, version 3.3.3-SNAPSHOT +# +# Optional ENV vars +# ----------------- +# JAVA_HOME - location of a JDK home dir, required when download maven via java source +# MVNW_REPOURL - repo url base for downloading maven distribution +# MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +# MVNW_VERBOSE - true: enable verbose log; debug: trace the mvnw script; others: silence the output +# ---------------------------------------------------------------------------- + +set -euf +[ "${MVNW_VERBOSE-}" != debug ] || set -x + +# OS specific support. +native_path() { printf %s\\n "$1"; } +case "$(uname)" in +CYGWIN* | MINGW*) + [ -z "${JAVA_HOME-}" ] || JAVA_HOME="$(cygpath --unix "$JAVA_HOME")" + native_path() { cygpath --path --windows "$1"; } + ;; +esac + +# set JAVACMD and JAVACCMD +set_java_home() { + # For Cygwin and MinGW, ensure paths are in Unix format before anything is touched + if [ -n "${JAVA_HOME-}" ]; then + if [ -x "$JAVA_HOME/jre/sh/java" ]; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + JAVACCMD="$JAVA_HOME/jre/sh/javac" + else + JAVACMD="$JAVA_HOME/bin/java" + JAVACCMD="$JAVA_HOME/bin/javac" + + if [ ! -x "$JAVACMD" ] || [ ! -x "$JAVACCMD" ]; then + echo "The JAVA_HOME environment variable is not defined correctly, so mvnw cannot run." >&2 + echo "JAVA_HOME is set to \"$JAVA_HOME\", but \"\$JAVA_HOME/bin/java\" or \"\$JAVA_HOME/bin/javac\" does not exist." >&2 + return 1 + fi + fi + else + JAVACMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v java + )" || : + JAVACCMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v javac + )" || : + + if [ ! -x "${JAVACMD-}" ] || [ ! -x "${JAVACCMD-}" ]; then + echo "The java/javac command does not exist in PATH nor is JAVA_HOME set, so mvnw cannot run." >&2 + return 1 + fi + fi +} + +# hash string like Java String::hashCode +hash_string() { + str="${1:-}" h=0 + while [ -n "$str" ]; do + char="${str%"${str#?}"}" + h=$(((h * 31 + $(LC_CTYPE=C printf %d "'$char")) % 4294967296)) + str="${str#?}" + done + printf %x\\n $h +} + +verbose() { :; } +[ "${MVNW_VERBOSE-}" != true ] || verbose() { printf %s\\n "${1-}"; } + +die() { + printf %s\\n "$1" >&2 + exit 1 +} + +trim() { + # MWRAPPER-139: + # Trims trailing and leading whitespace, carriage returns, tabs, and linefeeds. + # Needed for removing poorly interpreted newline sequences when running in more + # exotic environments such as mingw bash on Windows. + printf "%s" "${1}" | tr -d '[:space:]' +} + +# parse distributionUrl and optional distributionSha256Sum, requires .mvn/wrapper/maven-wrapper.properties +while IFS="=" read -r key value; do + case "${key-}" in + distributionUrl) distributionUrl=$(trim "${value-}") ;; + distributionSha256Sum) distributionSha256Sum=$(trim "${value-}") ;; + esac +done <"${0%/*}/.mvn/wrapper/maven-wrapper.properties" +[ -n "${distributionUrl-}" ] || die "cannot read distributionUrl property in ${0%/*}/.mvn/wrapper/maven-wrapper.properties" + +case "${distributionUrl##*/}" in +maven-mvnd-*bin.*) + MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ + case "${PROCESSOR_ARCHITECTURE-}${PROCESSOR_ARCHITEW6432-}:$(uname -a)" in + *AMD64:CYGWIN* | *AMD64:MINGW*) distributionPlatform=windows-amd64 ;; + :Darwin*x86_64) distributionPlatform=darwin-amd64 ;; + :Darwin*arm64) distributionPlatform=darwin-aarch64 ;; + :Linux*x86_64*) distributionPlatform=linux-amd64 ;; + *) + echo "Cannot detect native platform for mvnd on $(uname)-$(uname -m), use pure java version" >&2 + distributionPlatform=linux-amd64 + ;; + esac + distributionUrl="${distributionUrl%-bin.*}-$distributionPlatform.zip" + ;; +maven-mvnd-*) MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ ;; +*) MVN_CMD="mvn${0##*/mvnw}" _MVNW_REPO_PATTERN=/org/apache/maven/ ;; +esac + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +[ -z "${MVNW_REPOURL-}" ] || distributionUrl="$MVNW_REPOURL$_MVNW_REPO_PATTERN${distributionUrl#*"$_MVNW_REPO_PATTERN"}" +distributionUrlName="${distributionUrl##*/}" +distributionUrlNameMain="${distributionUrlName%.*}" +distributionUrlNameMain="${distributionUrlNameMain%-bin}" +MAVEN_USER_HOME="${MAVEN_USER_HOME:-${HOME}/.m2}" +MAVEN_HOME="${MAVEN_USER_HOME}/wrapper/dists/${distributionUrlNameMain-}/$(hash_string "$distributionUrl")" + +exec_maven() { + unset MVNW_VERBOSE MVNW_USERNAME MVNW_PASSWORD MVNW_REPOURL || : + exec "$MAVEN_HOME/bin/$MVN_CMD" "$@" || die "cannot exec $MAVEN_HOME/bin/$MVN_CMD" +} + +if [ -d "$MAVEN_HOME" ]; then + verbose "found existing MAVEN_HOME at $MAVEN_HOME" + exec_maven "$@" +fi + +case "${distributionUrl-}" in +*?-bin.zip | *?maven-mvnd-?*-?*.zip) ;; +*) die "distributionUrl is not valid, must match *-bin.zip or maven-mvnd-*.zip, but found '${distributionUrl-}'" ;; +esac + +# prepare tmp dir +if TMP_DOWNLOAD_DIR="$(mktemp -d)" && [ -d "$TMP_DOWNLOAD_DIR" ]; then + clean() { rm -rf -- "$TMP_DOWNLOAD_DIR"; } + trap clean HUP INT TERM EXIT +else + die "cannot create temp dir" +fi + +mkdir -p -- "${MAVEN_HOME%/*}" + +# Download and Install Apache Maven +verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +verbose "Downloading from: $distributionUrl" +verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +# select .zip or .tar.gz +if ! command -v unzip >/dev/null; then + distributionUrl="${distributionUrl%.zip}.tar.gz" + distributionUrlName="${distributionUrl##*/}" +fi + +# verbose opt +__MVNW_QUIET_WGET=--quiet __MVNW_QUIET_CURL=--silent __MVNW_QUIET_UNZIP=-q __MVNW_QUIET_TAR='' +[ "${MVNW_VERBOSE-}" != true ] || __MVNW_QUIET_WGET='' __MVNW_QUIET_CURL='' __MVNW_QUIET_UNZIP='' __MVNW_QUIET_TAR=v + +# normalize http auth +case "${MVNW_PASSWORD:+has-password}" in +'') MVNW_USERNAME='' MVNW_PASSWORD='' ;; +has-password) [ -n "${MVNW_USERNAME-}" ] || MVNW_USERNAME='' MVNW_PASSWORD='' ;; +esac + +if [ -z "${MVNW_USERNAME-}" ] && command -v wget >/dev/null; then + verbose "Found wget ... using wget" + wget ${__MVNW_QUIET_WGET:+"$__MVNW_QUIET_WGET"} "$distributionUrl" -O "$TMP_DOWNLOAD_DIR/$distributionUrlName" || die "wget: Failed to fetch $distributionUrl" +elif [ -z "${MVNW_USERNAME-}" ] && command -v curl >/dev/null; then + verbose "Found curl ... using curl" + curl ${__MVNW_QUIET_CURL:+"$__MVNW_QUIET_CURL"} -f -L -o "$TMP_DOWNLOAD_DIR/$distributionUrlName" "$distributionUrl" || die "curl: Failed to fetch $distributionUrl" +elif set_java_home; then + verbose "Falling back to use Java to download" + javaSource="$TMP_DOWNLOAD_DIR/Downloader.java" + targetZip="$TMP_DOWNLOAD_DIR/$distributionUrlName" + cat >"$javaSource" <<-END + public class Downloader extends java.net.Authenticator + { + protected java.net.PasswordAuthentication getPasswordAuthentication() + { + return new java.net.PasswordAuthentication( System.getenv( "MVNW_USERNAME" ), System.getenv( "MVNW_PASSWORD" ).toCharArray() ); + } + public static void main( String[] args ) throws Exception + { + setDefault( new Downloader() ); + java.nio.file.Files.copy( java.net.URI.create( args[0] ).toURL().openStream(), java.nio.file.Paths.get( args[1] ).toAbsolutePath().normalize() ); + } + } + END + # For Cygwin/MinGW, switch paths to Windows format before running javac and java + verbose " - Compiling Downloader.java ..." + "$(native_path "$JAVACCMD")" "$(native_path "$javaSource")" || die "Failed to compile Downloader.java" + verbose " - Running Downloader.java ..." + "$(native_path "$JAVACMD")" -cp "$(native_path "$TMP_DOWNLOAD_DIR")" Downloader "$distributionUrl" "$(native_path "$targetZip")" +fi + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +if [ -n "${distributionSha256Sum-}" ]; then + distributionSha256Result=false + if [ "$MVN_CMD" = mvnd.sh ]; then + echo "Checksum validation is not supported for maven-mvnd." >&2 + echo "Please disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + elif command -v sha256sum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | sha256sum -c >/dev/null 2>&1; then + distributionSha256Result=true + fi + elif command -v shasum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | shasum -a 256 -c >/dev/null 2>&1; then + distributionSha256Result=true + fi + else + echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." >&2 + echo "Please install either command, or disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + fi + if [ $distributionSha256Result = false ]; then + echo "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised." >&2 + echo "If you updated your Maven version, you need to update the specified distributionSha256Sum property." >&2 + exit 1 + fi +fi + +# unzip and move +if command -v unzip >/dev/null; then + unzip ${__MVNW_QUIET_UNZIP:+"$__MVNW_QUIET_UNZIP"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -d "$TMP_DOWNLOAD_DIR" || die "failed to unzip" +else + tar xzf${__MVNW_QUIET_TAR:+"$__MVNW_QUIET_TAR"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -C "$TMP_DOWNLOAD_DIR" || die "failed to untar" +fi +printf %s\\n "$distributionUrl" >"$TMP_DOWNLOAD_DIR/$distributionUrlNameMain/mvnw.url" +mv -- "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" "$MAVEN_HOME" || [ -d "$MAVEN_HOME" ] || die "fail to move MAVEN_HOME" + +clean || : +exec_maven "$@" diff --git a/mvnw.cmd b/mvnw.cmd new file mode 100644 index 000000000..f6af60a91 --- /dev/null +++ b/mvnw.cmd @@ -0,0 +1,149 @@ +<# : batch portion +@REM ---------------------------------------------------------------------------- +@REM Licensed to the Apache Software Foundation (ASF) under one +@REM or more contributor license agreements. See the NOTICE file +@REM distributed with this work for additional information +@REM regarding copyright ownership. The ASF licenses this file +@REM to you under the Apache License, Version 2.0 (the +@REM "License"); you may not use this file except in compliance +@REM with the License. You may obtain a copy of the License at +@REM +@REM http://www.apache.org/licenses/LICENSE-2.0 +@REM +@REM Unless required by applicable law or agreed to in writing, +@REM software distributed under the License is distributed on an +@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +@REM KIND, either express or implied. See the License for the +@REM specific language governing permissions and limitations +@REM under the License. +@REM ---------------------------------------------------------------------------- + +@REM ---------------------------------------------------------------------------- +@REM Apache Maven Wrapper startup batch script, version 3.3.3-SNAPSHOT +@REM +@REM Optional ENV vars +@REM MVNW_REPOURL - repo url base for downloading maven distribution +@REM MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +@REM MVNW_VERBOSE - true: enable verbose log; others: silence the output +@REM ---------------------------------------------------------------------------- + +@IF "%__MVNW_ARG0_NAME__%"=="" (SET __MVNW_ARG0_NAME__=%~nx0) +@SET __MVNW_CMD__= +@SET __MVNW_ERROR__= +@SET __MVNW_PSMODULEP_SAVE=%PSModulePath% +@SET PSModulePath= +@FOR /F "usebackq tokens=1* delims==" %%A IN (`powershell -noprofile "& {$scriptDir='%~dp0'; $script='%__MVNW_ARG0_NAME__%'; icm -ScriptBlock ([Scriptblock]::Create((Get-Content -Raw '%~f0'))) -NoNewScope}"`) DO @( + IF "%%A"=="MVN_CMD" (set __MVNW_CMD__=%%B) ELSE IF "%%B"=="" (echo %%A) ELSE (echo %%A=%%B) +) +@SET PSModulePath=%__MVNW_PSMODULEP_SAVE% +@SET __MVNW_PSMODULEP_SAVE= +@SET __MVNW_ARG0_NAME__= +@SET MVNW_USERNAME= +@SET MVNW_PASSWORD= +@IF NOT "%__MVNW_CMD__%"=="" (%__MVNW_CMD__% %*) +@echo Cannot start maven from wrapper >&2 && exit /b 1 +@GOTO :EOF +: end batch / begin powershell #> + +$ErrorActionPreference = "Stop" +if ($env:MVNW_VERBOSE -eq "true") { + $VerbosePreference = "Continue" +} + +# calculate distributionUrl, requires .mvn/wrapper/maven-wrapper.properties +$distributionUrl = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionUrl +if (!$distributionUrl) { + Write-Error "cannot read distributionUrl property in $scriptDir/.mvn/wrapper/maven-wrapper.properties" +} + +switch -wildcard -casesensitive ( $($distributionUrl -replace '^.*/','') ) { + "maven-mvnd-*" { + $USE_MVND = $true + $distributionUrl = $distributionUrl -replace '-bin\.[^.]*$',"-windows-amd64.zip" + $MVN_CMD = "mvnd.cmd" + break + } + default { + $USE_MVND = $false + $MVN_CMD = $script -replace '^mvnw','mvn' + break + } +} + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +if ($env:MVNW_REPOURL) { + $MVNW_REPO_PATTERN = if ($USE_MVND) { "/org/apache/maven/" } else { "/maven/mvnd/" } + $distributionUrl = "$env:MVNW_REPOURL$MVNW_REPO_PATTERN$($distributionUrl -replace '^.*'+$MVNW_REPO_PATTERN,'')" +} +$distributionUrlName = $distributionUrl -replace '^.*/','' +$distributionUrlNameMain = $distributionUrlName -replace '\.[^.]*$','' -replace '-bin$','' +$MAVEN_HOME_PARENT = "$HOME/.m2/wrapper/dists/$distributionUrlNameMain" +if ($env:MAVEN_USER_HOME) { + $MAVEN_HOME_PARENT = "$env:MAVEN_USER_HOME/wrapper/dists/$distributionUrlNameMain" +} +$MAVEN_HOME_NAME = ([System.Security.Cryptography.MD5]::Create().ComputeHash([byte[]][char[]]$distributionUrl) | ForEach-Object {$_.ToString("x2")}) -join '' +$MAVEN_HOME = "$MAVEN_HOME_PARENT/$MAVEN_HOME_NAME" + +if (Test-Path -Path "$MAVEN_HOME" -PathType Container) { + Write-Verbose "found existing MAVEN_HOME at $MAVEN_HOME" + Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" + exit $? +} + +if (! $distributionUrlNameMain -or ($distributionUrlName -eq $distributionUrlNameMain)) { + Write-Error "distributionUrl is not valid, must end with *-bin.zip, but found $distributionUrl" +} + +# prepare tmp dir +$TMP_DOWNLOAD_DIR_HOLDER = New-TemporaryFile +$TMP_DOWNLOAD_DIR = New-Item -Itemtype Directory -Path "$TMP_DOWNLOAD_DIR_HOLDER.dir" +$TMP_DOWNLOAD_DIR_HOLDER.Delete() | Out-Null +trap { + if ($TMP_DOWNLOAD_DIR.Exists) { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } + } +} + +New-Item -Itemtype Directory -Path "$MAVEN_HOME_PARENT" -Force | Out-Null + +# Download and Install Apache Maven +Write-Verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +Write-Verbose "Downloading from: $distributionUrl" +Write-Verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +$webclient = New-Object System.Net.WebClient +if ($env:MVNW_USERNAME -and $env:MVNW_PASSWORD) { + $webclient.Credentials = New-Object System.Net.NetworkCredential($env:MVNW_USERNAME, $env:MVNW_PASSWORD) +} +[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12 +$webclient.DownloadFile($distributionUrl, "$TMP_DOWNLOAD_DIR/$distributionUrlName") | Out-Null + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +$distributionSha256Sum = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionSha256Sum +if ($distributionSha256Sum) { + if ($USE_MVND) { + Write-Error "Checksum validation is not supported for maven-mvnd. `nPlease disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." + } + Import-Module $PSHOME\Modules\Microsoft.PowerShell.Utility -Function Get-FileHash + if ((Get-FileHash "$TMP_DOWNLOAD_DIR/$distributionUrlName" -Algorithm SHA256).Hash.ToLower() -ne $distributionSha256Sum) { + Write-Error "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised. If you updated your Maven version, you need to update the specified distributionSha256Sum property." + } +} + +# unzip and move +Expand-Archive "$TMP_DOWNLOAD_DIR/$distributionUrlName" -DestinationPath "$TMP_DOWNLOAD_DIR" | Out-Null +Rename-Item -Path "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" -NewName $MAVEN_HOME_NAME | Out-Null +try { + Move-Item -Path "$TMP_DOWNLOAD_DIR/$MAVEN_HOME_NAME" -Destination $MAVEN_HOME_PARENT | Out-Null +} catch { + if (! (Test-Path -Path "$MAVEN_HOME" -PathType Container)) { + Write-Error "fail to move MAVEN_HOME" + } +} finally { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } +} + +Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" diff --git a/pom.xml b/pom.xml index 8bb33946d..41d9aefc4 100644 --- a/pom.xml +++ b/pom.xml @@ -20,16 +20,16 @@ under the License. xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> - io.github.zentol.flink + org.apache.flink flink-connector-parent - 1.0 + 1.0.0 4.0.0 org.apache.flink flink-connector-kafka-parent - 4.0-SNAPSHOT + 3.4-SNAPSHOT Flink : Connectors : Kafka : Parent pom 2022 @@ -46,46 +46,67 @@ under the License. flink-connector-kafka flink-sql-connector-kafka flink-connector-kafka-e2e-tests + flink-python - 1.17-SNAPSHOT - 16.1 + 1.20.0 3.4.0 - 3.5.9 + 7.4.4 - 2.13.4.20221013 + 2.15.2 4.13.2 5.9.1 3.23.1 - 1.0.0 1.17.2 3.4.6 + 2.0.9 1.3 1.12.10 1.5.0 + 2.12 2.12.7 2.12.7 - 1.1.8.3 - 1.11.1 + 1.1.10.5 + 1.11.4 + 32.1.2-jre false - 1.16.0 + 1.17.0 1.7.32 2.17.1 flink-connector-kafka-parent + + + + -XX:+UseG1GC -Xms256m -XX:+IgnoreUnrecognizedVMOptions ${flink.connector.module.config} + - org.apache.flink - flink-shaded-force-shading - ${flink.shaded.version} + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + + + + com.fasterxml.jackson.datatype + jackson-datatype-jdk8 - - @@ -210,22 +231,6 @@ under the License. ${kafka.version} - - org.apache.zookeeper - zookeeper - ${zookeeper.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.xerial.snappy snappy-java @@ -250,6 +255,12 @@ under the License. 3.3.2 + + com.google.guava + guava + ${guava.version} + + org.apache.flink @@ -392,26 +403,33 @@ under the License. 2.1 + - org.testcontainers - testcontainers-bom - ${testcontainers.version} - pom - import + org.yaml + snakeyaml + 2.2 + - com.tngtech.archunit - archunit - ${archunit.version} - test + org.apache.commons + commons-compress + 1.26.1 + - com.tngtech.archunit - archunit-junit5 - ${archunit.version} - test + commons-io + commons-io + 2.15.1 + + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index 7daf1c336..25ef1cf96 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -38,6 +38,40 @@ appender.file.layout.type = PatternLayout appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n appender.file.createOnDemand = true +# Reduce most flink logs except for connector specific loggers +logger.flink.name = org.apache.flink +logger.flink.level = WARN +logger.flinkconnector.name = org.apache.flink.connector +logger.flinkconnector.level = INFO + +# Kafka producer and consumer level +logger.kafka.name = org.apache.kafka +logger.kafka.level = OFF + # suppress the irrelevant (wrong) warnings from the netty channel handler logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline logger.netty.level = ERROR + +# Logger configuration for containers, by default this is off +# If you want to investigate test failures, overwrite the level as above +logger.container.name = container +logger.container.level = OFF +logger.container.additivity = false # This prevents messages from being logged by the root logger +logger.container.appenderRef.containerappender.ref = ContainerLogger + +logger.kafkacontainer.name = container.kafka +logger.kafkacontainer.level = WARN + +logger.flinkcontainer.name = container.flink +logger.flinkcontainer.level = WARN + +logger.flinkenv.name = org.apache.flink.connector.testframe.container.FlinkContainerTestEnvironment +logger.flinkenv.level = WARN +logger.flinkenv.additivity = false # This prevents messages from being logged by the root logger +logger.flinkenv.appenderRef.containerappender.ref = ContainerLogger + +appender.containerappender.name = ContainerLogger +appender.containerappender.type = CONSOLE +appender.containerappender.target = SYSTEM_ERR +appender.containerappender.layout.type = PatternLayout +appender.containerappender.layout.pattern = [%c{1}] %m%n diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 2048fd186..7d1f9fd71 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -224,10 +224,6 @@ This file is based on the checkstyle file of Apache Beam. - - - -