diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 00000000..2e260749 --- /dev/null +++ b/.editorconfig @@ -0,0 +1,276 @@ +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 + +[*.xml] +indent_style = tab +indent_size = 4 + +[*.scala] +indent_style = space +indent_size = 2 + +[*.py] +indent_style = space +indent_size = 4 diff --git a/.gitignore b/.gitignore new file mode 100644 index 00000000..acbe2176 --- /dev/null +++ b/.gitignore @@ -0,0 +1,38 @@ +.eslintcache +.cache +scalastyle-output.xml +.classpath +.idea/* +!.idea/vcs.xml +.metadata +.settings +.project +.version.properties +filter.properties +logs.zip +.mvn/wrapper/*.jar +target +tmp +*.class +*.iml +*.swp +*.jar +*.zip +*.log +*.pyc +.DS_Store +build-target +atlassian-ide-plugin.xml +out/ +/docs/api +/docs/.bundle +/docs/.rubydeps +/docs/ruby2/.bundle +/docs/ruby2/.rubydeps +/docs/.jekyll-metadata +*.ipr +*.iws +tools/flink +tools/flink-* +tools/releasing/release +tools/japicmp-output diff --git a/LICENSE b/LICENSE new file mode 100644 index 00000000..261eeb9e --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/NOTICE b/NOTICE new file mode 100644 index 00000000..30367127 --- /dev/null +++ b/NOTICE @@ -0,0 +1,17 @@ +Apache Flink +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby +granted, provided that this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING +ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, +DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, +WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE +USE OR PERFORMANCE OF THIS SOFTWARE. + + + diff --git a/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 b/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 new file mode 100644 index 00000000..a215b58b --- /dev/null +++ b/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 @@ -0,0 +1,6 @@ +org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase 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 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 diff --git a/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d b/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d new file mode 100644 index 00000000..e69de29b diff --git a/flink-connector-elasticsearch-base/archunit-violations/stored.rules b/flink-connector-elasticsearch-base/archunit-violations/stored.rules new file mode 100644 index 00000000..76f9416d --- /dev/null +++ b/flink-connector-elasticsearch-base/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Tue Feb 22 12:16:59 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=de342dd1-c974-42c9-8f64-ef182ba8c56d +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=dd583797-83e1-414c-a38d-330773978813 diff --git a/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml new file mode 100644 index 00000000..a5afd009 --- /dev/null +++ b/flink-connector-elasticsearch-base/pom.xml @@ -0,0 +1,199 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + .. + + + flink-connector-elasticsearch-base + Flink : Connectors : Elasticsearch base + + jar + + + + 7.10.2 + + + + + + + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + true + + + + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + + + + org.ow2.asm + * + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + org.apache.httpcomponents + httpcore-nio + + + + + + + + org.testcontainers + elasticsearch + test + + + + org.apache.flink + flink-test-utils + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + org.apache.flink + flink-runtime + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test + test-jar + + + + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${flink.version} + test + + + + + + org.apache.logging.log4j + log4j-api + provided + + + + org.apache.logging.log4j + log4j-core + test + + + + + + org.apache.flink + flink-architecture-tests-test + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java new file mode 100644 index 00000000..f4a76989 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java @@ -0,0 +1,36 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.function.TriFunction; + +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.Serializable; + +@Internal +interface BulkProcessorBuilderFactory + extends Serializable, + TriFunction< + RestHighLevelClient, + BulkProcessorConfig, + BulkProcessor.Listener, + BulkProcessor.Builder> {} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java new file mode 100644 index 00000000..35fa1ecb --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java @@ -0,0 +1,72 @@ +/* + * 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.elasticsearch.sink; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class BulkProcessorConfig implements Serializable { + + private final int bulkFlushMaxActions; + private final int bulkFlushMaxMb; + private final long bulkFlushInterval; + private final FlushBackoffType flushBackoffType; + private final int bulkFlushBackoffRetries; + private final long bulkFlushBackOffDelay; + + BulkProcessorConfig( + int bulkFlushMaxActions, + int bulkFlushMaxMb, + long bulkFlushInterval, + FlushBackoffType flushBackoffType, + int bulkFlushBackoffRetries, + long bulkFlushBackOffDelay) { + this.bulkFlushMaxActions = bulkFlushMaxActions; + this.bulkFlushMaxMb = bulkFlushMaxMb; + this.bulkFlushInterval = bulkFlushInterval; + this.flushBackoffType = checkNotNull(flushBackoffType); + this.bulkFlushBackoffRetries = bulkFlushBackoffRetries; + this.bulkFlushBackOffDelay = bulkFlushBackOffDelay; + } + + public int getBulkFlushMaxActions() { + return bulkFlushMaxActions; + } + + public int getBulkFlushMaxMb() { + return bulkFlushMaxMb; + } + + public long getBulkFlushInterval() { + return bulkFlushInterval; + } + + public FlushBackoffType getFlushBackoffType() { + return flushBackoffType; + } + + public int getBulkFlushBackoffRetries() { + return bulkFlushBackoffRetries; + } + + public long getBulkFlushBackOffDelay() { + return bulkFlushBackOffDelay; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java new file mode 100644 index 00000000..7ef9eab2 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.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.elasticsearch.sink; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; + +import java.util.function.BiConsumer; + +/** + * {@link BulkRequestConsumerFactory} is used to bridge incompatible Elasticsearch Java API calls + * across different Elasticsearch versions. + */ +@Internal +interface BulkRequestConsumerFactory + extends BiConsumer> {} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java new file mode 100644 index 00000000..950f8627 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import org.elasticsearch.action.ActionRequest; + +/** + * Creates none or multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + *

This is used by sinks to prepare elements for sending them to Elasticsearch. + * + *

Example: + * + *

{@code
+ * private static class TestElasticsearchEmitter implements ElasticsearchEmitter> {
+ *
+ *     public IndexRequest createIndexRequest(Tuple2 element) {
+ *         Map document = new HashMap<>();
+ * 		   document.put("data", element.f1);
+ *
+ * 	       return Requests.indexRequest()
+ * 		       .index("my-index")
+ * 			   .type("my-type")
+ * 			   .id(element.f0.toString())
+ * 			   .source(document);
+ *     }
+ *
+ * 	   public void emit(Tuple2 element, RequestIndexer indexer) {
+ * 	       indexer.add(createIndexRequest(element));
+ *     }
+ * }
+ *
+ * }
+ * + * @param The type of the element handled by this {@link ElasticsearchEmitter} + */ +@PublicEvolving +public interface ElasticsearchEmitter extends Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param context to access additional information about the record + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void emit(T element, SinkWriter.Context context, RequestIndexer indexer); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java new file mode 100644 index 00000000..efe6dc24 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -0,0 +1,97 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.base.DeliveryGuarantee; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Flink Sink to insert or update data in an Elasticsearch index. The sink supports the following + * delivery guarantees. + * + *
    + *
  • {@link DeliveryGuarantee#NONE} does not provide any guarantees: actions are flushed to + * Elasticsearch only depending on the configurations of the bulk processor. In case of a + * failure, it might happen that actions are lost if the bulk processor still has buffered + * actions. + *
  • {@link DeliveryGuarantee#AT_LEAST_ONCE} on a checkpoint the sink will wait until all + * buffered actions are flushed to and acknowledged by Elasticsearch. No actions will be lost + * but actions might be sent to Elasticsearch multiple times when Flink restarts. These + * additional requests may cause inconsistent data in ElasticSearch right after the restart, + * but eventually everything will be consistent again. + *
+ * + * @param type of the records converted to Elasticsearch actions + * @see ElasticsearchSinkBuilderBase on how to construct a ElasticsearchSink + */ +@PublicEvolving +public class ElasticsearchSink implements Sink { + + private final List hosts; + private final ElasticsearchEmitter emitter; + private final BulkProcessorConfig buildBulkProcessorConfig; + private final BulkProcessorBuilderFactory bulkProcessorBuilderFactory; + private final NetworkClientConfig networkClientConfig; + private final DeliveryGuarantee deliveryGuarantee; + + ElasticsearchSink( + List hosts, + ElasticsearchEmitter emitter, + DeliveryGuarantee deliveryGuarantee, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + BulkProcessorConfig buildBulkProcessorConfig, + NetworkClientConfig networkClientConfig) { + this.hosts = checkNotNull(hosts); + this.bulkProcessorBuilderFactory = checkNotNull(bulkProcessorBuilderFactory); + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + this.emitter = checkNotNull(emitter); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.buildBulkProcessorConfig = checkNotNull(buildBulkProcessorConfig); + this.networkClientConfig = checkNotNull(networkClientConfig); + } + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new ElasticsearchWriter<>( + hosts, + emitter, + deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE, + buildBulkProcessorConfig, + bulkProcessorBuilderFactory, + networkClientConfig, + context.metricGroup(), + context.getMailboxExecutor()); + } + + @VisibleForTesting + DeliveryGuarantee getDeliveryGuarantee() { + return deliveryGuarantee; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java new file mode 100644 index 00000000..8543255d --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -0,0 +1,342 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Base builder to construct a {@link ElasticsearchSink}. + * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public abstract class ElasticsearchSinkBuilderBase< + IN, B extends ElasticsearchSinkBuilderBase> { + + private int bulkFlushMaxActions = 1000; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + private List hosts; + protected ElasticsearchEmitter emitter; + private String username; + private String password; + private String connectionPathPrefix; + private Integer connectionTimeout; + private Integer connectionRequestTimeout; + private Integer socketTimeout; + + protected ElasticsearchSinkBuilderBase() {} + + @SuppressWarnings("unchecked") + protected > S self() { + return (S) this; + } + + /** + * Sets the emitter which is invoked on every record to convert it to Elasticsearch actions. + * + * @param emitter to process records into Elasticsearch actions. + * @return this builder + */ + public ElasticsearchSinkBuilderBase setEmitter( + ElasticsearchEmitter emitter) { + checkNotNull(emitter); + checkState( + InstantiationUtil.isSerializable(emitter), + "The elasticsearch emitter must be serializable."); + + final ElasticsearchSinkBuilderBase self = self(); + self.emitter = emitter; + return self; + } + + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + * @return this builder + */ + public B setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return self(); + } + + /** + * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link + * DeliveryGuarantee#NONE} + * + * @param deliveryGuarantee which describes the record emission behaviour + * @return this builder + */ + public B setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { + checkState( + deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, + "Elasticsearch sink does not support the EXACTLY_ONCE guarantee."); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + return self(); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. The default flush size 1000. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + * @return this builder + */ + public B setBulkFlushMaxActions(int numMaxActions) { + checkState( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkFlushMaxActions = numMaxActions; + return self(); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + * @return this builder + */ + public B setBulkFlushMaxSizeMb(int maxSizeMb) { + checkState( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkFlushMaxMb = maxSizeMb; + return self(); + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + * @return this builder + */ + public B setBulkFlushInterval(long intervalMillis) { + checkState( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than " + + "or equal to 0."); + this.bulkFlushInterval = intervalMillis; + return self(); + } + + /** + * Sets the type of back off to use when flushing bulk requests. The default bulk flush back off + * type is {@link FlushBackoffType#NONE}. + * + *

Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + * @return this builder + */ + public B setBulkFlushBackoffStrategy( + FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) { + this.bulkFlushBackoffType = checkNotNull(flushBackoffType); + checkState( + flushBackoffType != FlushBackoffType.NONE, + "FlushBackoffType#NONE does not require a configuration it is the default, retries and delay are ignored."); + checkState(maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + this.bulkFlushBackoffRetries = maxRetries; + checkState( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger " + + "than or equal to 0."); + this.bulkFlushBackOffDelay = delayMillis; + return self(); + } + + /** + * Sets the username used to authenticate the connection with the Elasticsearch cluster. + * + * @param username of the Elasticsearch cluster user + * @return this builder + */ + public B setConnectionUsername(String username) { + checkNotNull(username); + this.username = username; + return self(); + } + + /** + * Sets the password used to authenticate the conection with the Elasticsearch cluster. + * + * @param password of the Elasticsearch cluster user + * @return this builder + */ + public B setConnectionPassword(String password) { + checkNotNull(password); + this.password = password; + return self(); + } + + /** + * Sets a prefix which used for every REST communication to the Elasticsearch cluster. + * + * @param prefix for the communication + * @return this builder + */ + public B setConnectionPathPrefix(String prefix) { + checkNotNull(prefix); + this.connectionPathPrefix = prefix; + return self(); + } + + /** + * Sets the timeout for requesting the connection of the Elasticsearch cluster from the + * connection manager. + * + * @param timeout for the connection request + * @return this builder + */ + public B setConnectionRequestTimeout(int timeout) { + checkState(timeout >= 0, "Connection request timeout must be larger than or equal to 0."); + this.connectionRequestTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for establishing a connection of the Elasticsearch cluster. + * + * @param timeout for the connection + * @return this builder + */ + public B setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for waiting for data or, put differently, a maximum period inactivity + * between two consecutive data packets. + * + * @param timeout for the socket + * @return this builder + */ + public B setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return self(); + } + + protected abstract BulkProcessorBuilderFactory getBulkProcessorBuilderFactory(); + + /** + * Constructs the {@link ElasticsearchSink} with the properties configured this builder. + * + * @return {@link ElasticsearchSink} + */ + public ElasticsearchSink build() { + checkNotNull(emitter); + checkNotNull(hosts); + + NetworkClientConfig networkClientConfig = buildNetworkClientConfig(); + BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig(); + + BulkProcessorBuilderFactory bulkProcessorBuilderFactory = getBulkProcessorBuilderFactory(); + ClosureCleaner.clean( + bulkProcessorBuilderFactory, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); + + return new ElasticsearchSink<>( + hosts, + emitter, + deliveryGuarantee, + bulkProcessorBuilderFactory, + bulkProcessorConfig, + networkClientConfig); + } + + private NetworkClientConfig buildNetworkClientConfig() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + + return new NetworkClientConfig( + username, + password, + connectionPathPrefix, + connectionRequestTimeout, + connectionTimeout, + socketTimeout); + } + + private BulkProcessorConfig buildBulkProcessorConfig() { + return new BulkProcessorConfig( + bulkFlushMaxActions, + bulkFlushMaxMb, + bulkFlushInterval, + bulkFlushBackoffType, + bulkFlushBackoffRetries, + bulkFlushBackOffDelay); + } + + @Override + public String toString() { + return "ElasticsearchSinkBuilder{" + + "bulkFlushMaxActions=" + + bulkFlushMaxActions + + ", bulkFlushMaxMb=" + + bulkFlushMaxMb + + ", bulkFlushInterval=" + + bulkFlushInterval + + ", bulkFlushBackoffType=" + + bulkFlushBackoffType + + ", bulkFlushBackoffRetries=" + + bulkFlushBackoffRetries + + ", bulkFlushBackOffDelay=" + + bulkFlushBackOffDelay + + ", deliveryGuarantee=" + + deliveryGuarantee + + ", hosts=" + + hosts + + ", emitter=" + + emitter + + ", username='" + + username + + '\'' + + ", password='" + + password + + '\'' + + ", connectionPathPrefix='" + + connectionPathPrefix + + '\'' + + '}'; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java new file mode 100644 index 00000000..fa8ed67c --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -0,0 +1,330 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; + +import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed; +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter implements SinkWriter { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchEmitter emitter; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long ackTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param emitter converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param bulkProcessorBuilderFactory configuring the {@link BulkProcessor}'s builder + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + ElasticsearchWriter( + List hosts, + ElasticsearchEmitter emitter, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.emitter = checkNotNull(emitter); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(metricGroup.getNumRecordsSendCounter()); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + emitter.open(); + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to open the ElasticsearchEmitter", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + emitter.emit(element, context, requestIndexer); + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0 && (flushOnCheckpoint || endOfInput)) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + } + + @VisibleForTesting + void blockingFlushAllActions() throws InterruptedException { + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + } + + @Override + public void close() throws Exception { + closed = true; + emitter.close(); + bulkProcessor.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + if (networkClientConfig.getConnectionRequestTimeout() != null + || networkClientConfig.getConnectionTimeout() != null + || networkClientConfig.getSocketTimeout() != null) { + builder.setRequestConfigCallback( + requestConfigBuilder -> { + if (networkClientConfig.getConnectionRequestTimeout() != null) { + requestConfigBuilder.setConnectionRequestTimeout( + networkClientConfig.getConnectionRequestTimeout()); + } + if (networkClientConfig.getConnectionTimeout() != null) { + requestConfigBuilder.setConnectTimeout( + networkClientConfig.getConnectionTimeout()); + } + if (networkClientConfig.getSocketTimeout() != null) { + requestConfigBuilder.setSocketTimeout( + networkClientConfig.getSocketTimeout()); + } + return requestConfigBuilder; + }); + } + return builder; + } + + private BulkProcessor createBulkProcessor( + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + BulkProcessorConfig bulkProcessorConfig) { + + BulkProcessor.Builder builder = + bulkProcessorBuilderFactory.apply(client, bulkProcessorConfig, new BulkListener()); + + // This makes flush() blocking + builder.setConcurrentRequests(0); + + return builder.build(); + } + + private class BulkListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) { + LOG.info("Sending bulk of {} actions to Elasticsearch.", request.numberOfActions()); + lastSendTime = System.currentTimeMillis(); + numBytesOutCounter.inc(request.estimatedSizeInBytes()); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + ackTime = System.currentTimeMillis(); + enqueueActionInMailbox( + () -> extractFailures(request, response), "elasticsearchSuccessCallback"); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + enqueueActionInMailbox( + () -> { + throw new FlinkRuntimeException("Complete bulk has failed.", failure); + }, + "elasticsearchErrorCallback"); + } + } + + private void enqueueActionInMailbox( + ThrowingRunnable action, String actionName) { + // If the writer is cancelled before the last bulk response (i.e. no flush on checkpoint + // configured or shutdown without a final + // checkpoint) the mailbox might already be shutdown, so we should not enqueue any + // actions. + if (isClosed()) { + return; + } + mailboxExecutor.execute(action, actionName); + } + + private void extractFailures(BulkRequest request, BulkResponse response) { + if (!response.hasFailures()) { + pendingActions -= request.numberOfActions(); + return; + } + + Throwable chainedFailures = null; + for (int i = 0; i < response.getItems().length; i++) { + final BulkItemResponse itemResponse = response.getItems()[i]; + if (!itemResponse.isFailed()) { + continue; + } + final Throwable failure = itemResponse.getFailure().getCause(); + if (failure == null) { + continue; + } + final RestStatus restStatus = itemResponse.getFailure().getStatus(); + final DocWriteRequest actionRequest = request.requests().get(i); + + chainedFailures = + firstOrSuppressed( + wrapException(restStatus, failure, actionRequest), chainedFailures); + } + if (chainedFailures == null) { + return; + } + throw new FlinkRuntimeException(chainedFailures); + } + + private static Throwable wrapException( + RestStatus restStatus, Throwable rootFailure, DocWriteRequest actionRequest) { + if (restStatus == null) { + return new FlinkRuntimeException( + String.format("Single action %s of bulk request failed.", actionRequest), + rootFailure); + } else { + return new FlinkRuntimeException( + String.format( + "Single action %s of bulk request failed with status %s.", + actionRequest, restStatus.getStatus()), + rootFailure); + } + } + + private boolean isClosed() { + if (closed) { + LOG.warn("Writer was closed before all records were acknowledged by Elasticsearch."); + } + return closed; + } + + private class DefaultRequestIndexer implements RequestIndexer { + + private final Counter numRecordsSendCounter; + + public DefaultRequestIndexer(Counter numRecordsSendCounter) { + this.numRecordsSendCounter = checkNotNull(numRecordsSendCounter); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (final DeleteRequest deleteRequest : deleteRequests) { + numRecordsSendCounter.inc(); + pendingActions++; + bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (final IndexRequest indexRequest : indexRequests) { + numRecordsSendCounter.inc(); + pendingActions++; + bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (final UpdateRequest updateRequest : updateRequests) { + numRecordsSendCounter.inc(); + pendingActions++; + bulkProcessor.add(updateRequest); + } + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java new file mode 100644 index 00000000..1e001f04 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java @@ -0,0 +1,38 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Used to control whether the sink should retry failed requests at all or with which kind back off + * strategy. + */ +@PublicEvolving +public enum FlushBackoffType { + /** After every failure, it waits a configured time until the retries are exhausted. */ + CONSTANT, + /** + * After every failure, it waits initially the configured time and increases the waiting time + * exponentially until the retries are exhausted. + */ + EXPONENTIAL, + /** The failure is not retried. */ + NONE, +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java new file mode 100644 index 00000000..5ae05108 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.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.elasticsearch.sink; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class NetworkClientConfig implements Serializable { + + @Nullable private final String username; + @Nullable private final String password; + @Nullable private final String connectionPathPrefix; + @Nullable private final Integer connectionRequestTimeout; + @Nullable private final Integer connectionTimeout; + @Nullable private final Integer socketTimeout; + + NetworkClientConfig( + @Nullable String username, + @Nullable String password, + @Nullable String connectionPathPrefix, + @Nullable Integer connectionRequestTimeout, + @Nullable Integer connectionTimeout, + @Nullable Integer socketTimeout) { + this.username = username; + this.password = password; + this.connectionPathPrefix = connectionPathPrefix; + this.connectionRequestTimeout = connectionRequestTimeout; + this.connectionTimeout = connectionTimeout; + this.socketTimeout = socketTimeout; + } + + @Nullable + public String getUsername() { + return username; + } + + @Nullable + public String getPassword() { + return password; + } + + @Nullable + public Integer getConnectionRequestTimeout() { + return connectionRequestTimeout; + } + + @Nullable + public Integer getConnectionTimeout() { + return connectionTimeout; + } + + @Nullable + public Integer getSocketTimeout() { + return socketTimeout; + } + + @Nullable + public String getConnectionPathPrefix() { + return connectionPathPrefix; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java new file mode 100644 index 00000000..830fb6a3 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java @@ -0,0 +1,56 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +/** + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Elasticsearch cluster. + */ +@PublicEvolving +public interface RequestIndexer { + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000..2a6dd929 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.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.connector.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.time.format.DateTimeFormatter; + +/** Abstract class for time related {@link IndexGenerator}. */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java new file mode 100644 index 00000000..3bedea14 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java @@ -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. + */ + +package org.apache.flink.connector.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Elasticsearch base configuration. */ +@Internal +class ElasticsearchConfiguration { + protected final ReadableConfig config; + + ElasticsearchConfiguration(ReadableConfig config) { + this.config = checkNotNull(config); + } + + public int getBulkFlushMaxActions() { + return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION); + } + + public MemorySize getBulkFlushMaxByteSize() { + return config.get(BULK_FLUSH_MAX_SIZE_OPTION); + } + + public long getBulkFlushInterval() { + return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + } + + public DeliveryGuarantee getDeliveryGuarantee() { + return config.get(DELIVERY_GUARANTEE_OPTION); + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public Optional getBulkFlushBackoffType() { + return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public String getIndex() { + return config.get(INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); + } + + public Optional getConnectionRequestTimeout() { + return config.getOptional(CONNECTION_REQUEST_TIMEOUT); + } + + public Optional getConnectionTimeout() { + return config.getOptional(CONNECTION_TIMEOUT); + } + + public Optional getSocketTimeout() { + return config.getOptional(SOCKET_TIMEOUT); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(ElasticsearchConfiguration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + public Optional getParallelism() { + return config.getOptional(SINK_PARALLELISM); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java new file mode 100644 index 00000000..10ea0ae2 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -0,0 +1,148 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; + +import java.time.Duration; +import java.util.List; + +/** + * Base options for the Elasticsearch connector. Needs to be public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ +@PublicEvolving +public class ElasticsearchConnectorOptions { + + ElasticsearchConnectorOptions() {} + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + + public static final ConfigOption BULK_FLUSH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(FlushBackoffType.class) + .noDefaultValue() + .withDescription("Backoff strategy"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption CONNECTION_REQUEST_TIMEOUT = + ConfigOptions.key("connection.request-timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The timeout for requesting a connection from the connection manager."); + + public static final ConfigOption CONNECTION_TIMEOUT = + ConfigOptions.key("connection.timeout") + .durationType() + .noDefaultValue() + .withDescription("The timeout for establishing a connection."); + + public static final ConfigOption SOCKET_TIMEOUT = + ConfigOptions.key("socket.timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The socket timeout (SO_TIMEOUT) for waiting for data or, put differently," + + "a maximum period inactivity between two consecutive data packets."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + public static final ConfigOption DELIVERY_GUARANTEE_OPTION = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) + .withDescription("Optional delivery guarantee when committing."); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java new file mode 100644 index 00000000..0fd389bd --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -0,0 +1,228 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@Internal +class ElasticsearchDynamicSink implements DynamicTableSink { + + final EncodingFormat> format; + final DataType physicalRowDataType; + final List primaryKeyLogicalTypesWithIndex; + final ElasticsearchConfiguration config; + final ZoneId localTimeZoneId; + + final String summaryString; + final ElasticsearchSinkBuilderSupplier builderSupplier; + @Nullable final String documentType; + final boolean isDynamicIndexWithSystemTime; + + ElasticsearchDynamicSink( + EncodingFormat> format, + ElasticsearchConfiguration config, + List primaryKeyLogicalTypesWithIndex, + DataType physicalRowDataType, + String summaryString, + ElasticsearchSinkBuilderSupplier builderSupplier, + @Nullable String documentType, + ZoneId localTimeZoneId) { + this.format = checkNotNull(format); + this.physicalRowDataType = checkNotNull(physicalRowDataType); + this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex); + this.config = checkNotNull(config); + this.summaryString = checkNotNull(summaryString); + this.builderSupplier = checkNotNull(builderSupplier); + this.documentType = documentType; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + } + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + Function createKeyExtractor() { + return KeyExtractor.createKeyExtractor( + primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter()); + } + + IndexGenerator createIndexGenerator() { + return IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), + DataType.getFieldNames(physicalRowDataType), + DataType.getFieldDataTypes(physicalRowDataType), + localTimeZoneId); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + SerializationSchema format = + this.format.createRuntimeEncoder(context, physicalRowDataType); + + final RowElasticsearchEmitter rowElasticsearchEmitter = + new RowElasticsearchEmitter( + createIndexGenerator(), + format, + XContentType.JSON, + documentType, + createKeyExtractor()); + + ElasticsearchSinkBuilderBase builder = + builderSupplier.get(); + builder.setEmitter(rowElasticsearchEmitter); + builder.setHosts(config.getHosts().toArray(new HttpHost[0])); + builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb(config.getBulkFlushMaxByteSize().getMebiBytes()); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + + if (config.getBulkFlushBackoffType().isPresent()) { + FlushBackoffType backoffType = config.getBulkFlushBackoffType().get(); + int backoffMaxRetries = config.getBulkFlushBackoffRetries().get(); + long backoffDelayMs = config.getBulkFlushBackoffDelay().get(); + + builder.setBulkFlushBackoffStrategy(backoffType, backoffMaxRetries, backoffDelayMs); + } + + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + builder.setConnectionUsername(config.getUsername().get()); + } + + if (config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setConnectionPassword(config.getPassword().get()); + } + + if (config.getPathPrefix().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) { + builder.setConnectionPathPrefix(config.getPathPrefix().get()); + } + + if (config.getConnectionRequestTimeout().isPresent()) { + builder.setConnectionRequestTimeout( + (int) config.getConnectionRequestTimeout().get().getSeconds()); + } + + if (config.getConnectionTimeout().isPresent()) { + builder.setConnectionTimeout((int) config.getConnectionTimeout().get().getSeconds()); + } + + if (config.getSocketTimeout().isPresent()) { + builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds()); + } + + return SinkV2Provider.of(builder.build(), config.getParallelism().orElse(null)); + } + + @Override + public DynamicTableSink copy() { + return new ElasticsearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + physicalRowDataType, + summaryString, + builderSupplier, + documentType, + localTimeZoneId); + } + + @Override + public String asSummaryString() { + return summaryString; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchDynamicSink that = (ElasticsearchDynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(physicalRowDataType, that.physicalRowDataType) + && Objects.equals( + primaryKeyLogicalTypesWithIndex, that.primaryKeyLogicalTypesWithIndex) + && Objects.equals(config, that.config) + && Objects.equals(summaryString, that.summaryString) + && Objects.equals(builderSupplier, that.builderSupplier) + && Objects.equals(documentType, that.documentType); + } + + @Override + public int hashCode() { + return Objects.hash( + format, + physicalRowDataType, + primaryKeyLogicalTypesWithIndex, + config, + summaryString, + builderSupplier, + documentType); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java new file mode 100644 index 00000000..ed5e7f73 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -0,0 +1,257 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.elasticsearch.common.Strings.capitalize; + +/** A {@link DynamicTableSinkFactory} for discovering ElasticsearchDynamicSink. */ +@Internal +abstract class ElasticsearchDynamicSinkFactoryBase implements DynamicTableSinkFactory { + + private final String factoryIdentifier; + private final ElasticsearchSinkBuilderSupplier sinkBuilderSupplier; + + public ElasticsearchDynamicSinkFactoryBase( + String factoryIdentifier, + ElasticsearchSinkBuilderSupplier sinkBuilderSupplier) { + this.factoryIdentifier = checkNotNull(factoryIdentifier); + this.sinkBuilderSupplier = checkNotNull(sinkBuilderSupplier); + } + + @Nullable + String getDocumentType(ElasticsearchConfiguration configuration) { + return null; // document type is only set in Elasticsearch versions < 7 + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + ElasticsearchConfiguration config = getConfiguration(helper); + helper.validate(); + validateConfiguration(config); + + return new ElasticsearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + context.getPhysicalRowDataType(), + capitalize(factoryIdentifier), + sinkBuilderSupplier, + getDocumentType(config), + getLocalTimeZoneId(context.getConfiguration())); + } + + ElasticsearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) { + return new ElasticsearchConfiguration(helper.getOptions()); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + void validateConfiguration(ElasticsearchConfiguration config) { + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize().getBytes(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLUSH_MAX_SIZE_OPTION.key(), + config.getBulkFlushMaxByteSize().toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + List getPrimaryKeyLogicalTypesWithIndex(Context context) { + DataType physicalRowDataType = context.getPhysicalRowDataType(); + int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); + if (primaryKeyIndexes.length != 0) { + DataType pkDataType = Projection.of(primaryKeyIndexes).project(physicalRowDataType); + + ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); + } + + ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); + return Arrays.stream(primaryKeyIndexes) + .mapToObj( + index -> { + Optional column = resolvedSchema.getColumn(index); + if (!column.isPresent()) { + throw new IllegalStateException( + String.format( + "No primary key column found with index '%s'.", + index)); + } + LogicalType logicalType = column.get().getDataType().getLogicalType(); + return new LogicalTypeWithIndex(index, logicalType); + }) + .collect(Collectors.toList()); + } + + @Override + public Set> requiredOptions() { + return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + KEY_DELIMITER_OPTION, + BULK_FLUSH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX_OPTION, + CONNECTION_REQUEST_TIMEOUT, + CONNECTION_TIMEOUT, + SOCKET_TIMEOUT, + FORMAT_OPTION, + DELIVERY_GUARANTEE_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION, + SINK_PARALLELISM) + .collect(Collectors.toSet()); + } + + @Override + public Set> forwardOptions() { + return Stream.of( + HOSTS_OPTION, + INDEX_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION, + KEY_DELIMITER_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_MAX_SIZE_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX_OPTION, + CONNECTION_REQUEST_TIMEOUT, + CONNECTION_TIMEOUT, + SOCKET_TIMEOUT) + .collect(Collectors.toSet()); + } + + @Override + public String factoryIdentifier() { + return factoryIdentifier; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java new file mode 100644 index 00000000..70088021 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.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.elasticsearch.table; + +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; + +import java.util.function.Supplier; + +interface ElasticsearchSinkBuilderSupplier + extends Supplier> {} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java new file mode 100644 index 00000000..3d60f345 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** Utility methods for validating Elasticsearch properties. */ +@Internal +class ElasticsearchValidationUtils { + private static final Set ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DATE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME); + } + + /** + * Checks that the table does not have a primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(DataType primaryKeyDataType) { + List fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType); + List illegalTypes = + fieldDataTypes.stream() + .map(DataType::getLogicalType) + .map( + logicalType -> { + if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(t -> !ALLOWED_PRIMARY_KEY_TYPES.contains(t)) + .collect(Collectors.toList()); + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.", + illegalTypes)); + } + } + + private ElasticsearchValidationUtils() {} +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java new file mode 100644 index 00000000..c34c9b2b --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java @@ -0,0 +1,39 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** This interface is responsible to generate index name from given {@link Row} record. */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} + + /** Generate index name according to the given row. */ + String generate(RowData row); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java new file mode 100644 index 00000000..8a89013e --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** Base class for {@link IndexGenerator}. */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java new file mode 100644 index 00000000..ec2a0069 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java @@ -0,0 +1,315 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator( + String index, + List fieldNames, + List dataTypes, + ZoneId localTimeZoneId) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + fieldNames.toArray(new String[0]), + dataTypes.toArray(new DataType[0]), + indexHelper, + localTimeZoneId); + } else { + return new StaticIndexGenerator(index); + } + } + + public static IndexGenerator createIndexGenerator( + String index, List fieldNames, List dataTypes) { + return createIndexGenerator(index, fieldNames, dataTypes, ZoneId.systemDefault()); + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper, + ZoneId localTimeZoneId) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) { + final String dateTimeFormat = + indexHelper.extractDateFormat( + index, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + return indexPrefix + .concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter)) + .concat(indexSuffix); + } + }; + } + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final Pattern dynamicIndexSystemTimeExtractPattern = + Pattern.compile( + ".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } + + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } + + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** Check generate dynamic index is from system time or not. */ + boolean checkIsDynamicIndexWithSystemTimeFormat(String index) { + return dynamicIndexSystemTimeExtractPattern.matcher(index).matches(); + } + + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java new file mode 100644 index 00000000..f369ee00 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java @@ -0,0 +1,97 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.function.SerializableFunction; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.List; + +/** An extractor for a Elasticsearch key from a {@link RowData}. */ +@Internal +class KeyExtractor implements SerializableFunction { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + public static SerializableFunction createKeyExtractor( + List primaryKeyTypesWithIndex, String keyDelimiter) { + if (!primaryKeyTypesWithIndex.isEmpty()) { + FieldFormatter[] formatters = + primaryKeyTypesWithIndex.stream() + .map( + logicalTypeWithIndex -> + toFormatter( + logicalTypeWithIndex.index, + logicalTypeWithIndex.logicalType)) + .toArray(FieldFormatter[]::new); + return new KeyExtractor(formatters, keyDelimiter); + } else { + return (row) -> null; + } + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java new file mode 100644 index 00000000..c43b7f48 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java @@ -0,0 +1,31 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.table.types.logical.LogicalType; + +class LogicalTypeWithIndex { + public final int index; + public final LogicalType logicalType; + + LogicalTypeWithIndex(int index, LogicalType logicalType) { + this.index = index; + this.logicalType = logicalType; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java new file mode 100644 index 00000000..e90e0bca --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java @@ -0,0 +1,107 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchEmitter; +import org.apache.flink.connector.elasticsearch.sink.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ +class RowElasticsearchEmitter implements ElasticsearchEmitter { + + private final IndexGenerator indexGenerator; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + @Nullable private final String documentType; + private final Function createKey; + + public RowElasticsearchEmitter( + IndexGenerator indexGenerator, + SerializationSchema serializationSchema, + XContentType contentType, + @Nullable String documentType, + Function createKey) { + this.indexGenerator = checkNotNull(indexGenerator); + this.serializationSchema = checkNotNull(serializationSchema); + this.contentType = checkNotNull(contentType); + this.documentType = documentType; + this.createKey = checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void emit(RowData element, SinkWriter.Context context, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + new UpdateRequest(indexGenerator.generate(row), documentType, key) + .doc(document, contentType) + .upsert(document, contentType); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + new IndexRequest(indexGenerator.generate(row), documentType) + .id(key) + .source(document, contentType); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = + new DeleteRequest(indexGenerator.generate(row), documentType, key); + indexer.add(deleteRequest); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java new file mode 100644 index 00000000..67e0a1fd --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** A static {@link IndexGenerator} which generate fixed index name. */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java new file mode 100644 index 00000000..8a3fed98 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -0,0 +1,82 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how + * failed {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing + * malformed documents, or simply requesting them to be sent to Elasticsearch again if the failure + * is only temporary. + * + *

Example: + * + *

{@code
+ * private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
+ *
+ * 	@Override
+ * 	void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
+ * 		if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) {
+ * 			// full queue; re-add document for indexing
+ * 			indexer.add(action);
+ * 		} else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) {
+ * 			// malformed document; simply drop request without failing sink
+ * 		} else {
+ * 			// for all other failures, fail the sink;
+ * 			// here the failure is simply rethrown, but users can also choose to throw custom exceptions
+ * 			throw failure;
+ * 		}
+ * 	}
+ * }
+ *
+ * }
+ * + *

The above example will let the sink re-add requests that failed due to queue capacity + * saturation and drop requests with malformed documents, without failing the sink. For all other + * failures, the sink will fail. + * + *

Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the + * exact type could not be retrieved through the older version Java client APIs (thus, the types + * will be general {@link Exception}s and only differ in the failure message). In this case, it is + * recommended to match on the provided REST status code. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface ActionRequestFailureHandler extends Serializable { + + /** + * Handle a failed {@link ActionRequest}. + * + * @param action the {@link ActionRequest} that failed due to the failure + * @param failure the cause of failure + * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved) + * @param indexer request indexer to re-add the failed action, if intended to do so + * @throws Throwable if the sink should fail on this failure, the implementation should rethrow + * the exception or a custom one + */ + void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable; +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java new file mode 100644 index 00000000..4f86ba73 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java @@ -0,0 +1,75 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.Collections; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests} + * before re-sending them to the Elasticsearch cluster upon request. + */ +@Internal +@NotThreadSafe +class BufferingNoOpRequestIndexer implements RequestIndexer { + + private ConcurrentLinkedQueue bufferedRequests; + + BufferingNoOpRequestIndexer() { + this.bufferedRequests = new ConcurrentLinkedQueue(); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + Collections.addAll(bufferedRequests, deleteRequests); + } + + @Override + public void add(IndexRequest... indexRequests) { + Collections.addAll(bufferedRequests, indexRequests); + } + + @Override + public void add(UpdateRequest... updateRequests) { + Collections.addAll(bufferedRequests, updateRequests); + } + + void processBufferedRequests(RequestIndexer actualIndexer) { + for (ActionRequest request : bufferedRequests) { + if (request instanceof IndexRequest) { + actualIndexer.add((IndexRequest) request); + } else if (request instanceof DeleteRequest) { + actualIndexer.add((DeleteRequest) request); + } else if (request instanceof UpdateRequest) { + actualIndexer.add((UpdateRequest) request); + } + } + + bufferedRequests.clear(); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java new file mode 100644 index 00000000..9b4f9fb2 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -0,0 +1,122 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * An {@link ElasticsearchApiCallBridge} is used to bridge incompatible Elasticsearch Java API calls + * across different versions. This includes calls to create Elasticsearch clients, handle failed + * item responses, etc. Any incompatible Elasticsearch Java APIs should be bridged using this + * interface. + * + *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since + * connecting via an embedded node is allowed, the call bridge will hold reference to the created + * embedded node. Each instance of the sink will hold exactly one instance of the call bridge, and + * state cleanup is performed when the sink is closed. + * + * @param The Elasticsearch client, that implements {@link AutoCloseable}. + */ +@Internal +public interface ElasticsearchApiCallBridge extends Serializable { + + /** + * Creates an Elasticsearch client implementing {@link AutoCloseable}. + * + * @param clientConfig The configuration to use when constructing the client. + * @return The created client. + */ + C createClient(Map clientConfig); + + /** + * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. + * + * @param client the Elasticsearch client. + * @param listener the bulk processor listener. + * @return the bulk processor builder. + */ + BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); + + /** + * Extracts the cause of failure of a bulk item action. + * + * @param bulkItemResponse the bulk item response to extract cause of failure + * @return the extracted {@link Throwable} from the response ({@code null} is the response is + * successful). + */ + @Nullable + Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + + /** + * Sets the bulk flush interval, in milliseconds on the provided {@link BulkProcessor.Builder}. + * The builder will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushIntervalMillis the flush interval in milliseconds. + */ + void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis); + + /** + * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. The builder + * will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user + * disabled backoff retries). + */ + void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); + + /** + * Verify the client connection by making a test request/ping to the Elasticsearch cluster. + * + *

Called by {@link ElasticsearchSinkBase#open(org.apache.flink.configuration.Configuration)} + * after creating the client. This makes sure the underlying client is closed if the connection + * is not successful and preventing thread leak. + * + * @param client the Elasticsearch client. + */ + void verifyClientConnection(C client) throws IOException; + + /** + * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary + * compatible. + */ + RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef); + + /** Perform any necessary state cleanup. */ + default void cleanup() { + // nothing to cleanup by default + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java new file mode 100644 index 00000000..9ed83090 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -0,0 +1,525 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +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.util.InstantiationUtil; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.rest.RestStatus; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all Flink Elasticsearch Sinks. + * + *

This class implements the common behaviour across Elasticsearch versions, such as the use of + * an internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before sending the + * requests to the cluster, as well as passing input records to the user provided {@link + * ElasticsearchSinkFunction} for processing. + * + *

The version specific API calls for different Elasticsearch versions should be defined by a + * concrete implementation of a {@link ElasticsearchApiCallBridge}, which is provided to the + * constructor of this class. This call bridge is used, for example, to create a Elasticsearch + * {@link Client}, handle failed item responses, etc. + * + * @param Type of the elements handled by this sink + * @param Type of the Elasticsearch client, which implements {@link AutoCloseable} + */ +@Internal +public abstract class ElasticsearchSinkBase extends RichSinkFunction + implements CheckpointedFunction { + + private static final long serialVersionUID = -1007596293618451942L; + + // ------------------------------------------------------------------------ + // Internal bulk processor configuration + // ------------------------------------------------------------------------ + + public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; + public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; + public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; + + /** Used to control whether the retry delay should increase exponentially or remain constant. */ + @PublicEvolving + public enum FlushBackoffType { + CONSTANT, + EXPONENTIAL + } + + /** + * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to + * resource constraints (i.e. the client's internal thread pool is full), the backoff policy + * decides how long the bulk processor will wait before the operation is retried internally. + * + *

This is a proxy for version specific backoff policies. + */ + public static class BulkFlushBackoffPolicy implements Serializable { + + private static final long serialVersionUID = -6022851996101826049L; + + // the default values follow the Elasticsearch default settings for BulkProcessor + private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL; + private int maxRetryCount = 8; + private long delayMillis = 50; + + public FlushBackoffType getBackoffType() { + return backoffType; + } + + public int getMaxRetryCount() { + return maxRetryCount; + } + + public long getDelayMillis() { + return delayMillis; + } + + public void setBackoffType(FlushBackoffType backoffType) { + this.backoffType = checkNotNull(backoffType); + } + + public void setMaxRetryCount(int maxRetryCount) { + checkArgument(maxRetryCount >= 0); + this.maxRetryCount = maxRetryCount; + } + + public void setDelayMillis(long delayMillis) { + checkArgument(delayMillis >= 0); + this.delayMillis = delayMillis; + } + } + + private final Integer bulkProcessorFlushMaxActions; + private final Integer bulkProcessorFlushMaxSizeMb; + private final Long bulkProcessorFlushIntervalMillis; + private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; + + // ------------------------------------------------------------------------ + // User-facing API and configuration + // ------------------------------------------------------------------------ + + /** + * The config map that contains configuration for the bulk flushing behaviours. + * + *

For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this + * config map would also contain Elasticsearch-shipped configuration, and therefore this config + * map would also be forwarded when creating the Elasticsearch client. + */ + private final Map userConfig; + + /** + * The function that is used to construct multiple {@link ActionRequest ActionRequests} from + * each incoming element. + */ + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ + private final ActionRequestFailureHandler failureHandler; + + /** + * If true, the producer will wait until all outstanding action requests have been sent to + * Elasticsearch. + */ + private boolean flushOnCheckpoint = true; + + /** + * Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest + * ActionRequests}. + */ + private transient RequestIndexer requestIndexer; + + /** + * Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed + * requests. + */ + private transient BufferingNoOpRequestIndexer failureRequestIndexer; + + // ------------------------------------------------------------------------ + // Internals for the Flink Elasticsearch Sink + // ------------------------------------------------------------------------ + + /** Call bridge for different version-specific. */ + private final ElasticsearchApiCallBridge callBridge; + + /** + * Number of pending action requests not yet acknowledged by Elasticsearch. This value is + * maintained only if {@link ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}. + * + *

This is incremented whenever the user adds (or re-adds through the {@link + * ActionRequestFailureHandler}) requests to the {@link RequestIndexer}. It is decremented for + * each completed request of a bulk request, in {@link BulkProcessor.Listener#afterBulk(long, + * BulkRequest, BulkResponse)} and {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, + * Throwable)}. + */ + private AtomicLong numPendingRequests = new AtomicLong(0); + + /** Elasticsearch client created using the call bridge. */ + private transient C client; + + /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ + private transient BulkProcessor bulkProcessor; + + /** + * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown + * in callbacks and the user considered it should fail the sink via the {@link + * ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. + * + *

Errors will be checked and rethrown before processing each input element, and when the + * sink is closed. + */ + private final AtomicReference failureThrowable = new AtomicReference<>(); + + public ElasticsearchSinkBase( + ElasticsearchApiCallBridge callBridge, + Map userConfig, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler) { + + this.callBridge = checkNotNull(callBridge); + this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); + this.failureHandler = checkNotNull(failureHandler); + // we eagerly check if the user-provided sink function and failure handler is serializable; + // otherwise, if they aren't serializable, users will merely get a non-informative error + // message + // "ElasticsearchSinkBase is not serializable" + + checkArgument( + InstantiationUtil.isSerializable(elasticsearchSinkFunction), + "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + + "The object probably contains or references non-serializable fields."); + + checkArgument( + InstantiationUtil.isSerializable(failureHandler), + "The implementation of the provided ActionRequestFailureHandler is not serializable. " + + "The object probably contains or references non-serializable fields."); + + // extract and remove bulk processor related configuration from the user-provided config, + // so that the resulting user config only contains configuration related to the + // Elasticsearch client. + + checkNotNull(userConfig); + + // copy config so we can remove entries without side-effects + userConfig = new HashMap<>(userConfig); + + ParameterTool params = ParameterTool.fromMap(userConfig); + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { + bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + } else { + bulkProcessorFlushMaxActions = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { + bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + } else { + bulkProcessorFlushMaxSizeMb = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { + bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + } else { + bulkProcessorFlushIntervalMillis = null; + } + + boolean bulkProcessorFlushBackoffEnable = + params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE); + + if (bulkProcessorFlushBackoffEnable) { + this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy(); + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) { + bulkProcessorFlushBackoffPolicy.setBackoffType( + FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE))); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) { + bulkProcessorFlushBackoffPolicy.setMaxRetryCount( + params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) { + bulkProcessorFlushBackoffPolicy.setDelayMillis( + params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY); + } + + } else { + bulkProcessorFlushBackoffPolicy = null; + } + + this.userConfig = userConfig; + } + + /** + * Disable flushing on checkpoint. When disabled, the sink will not wait for all pending action + * requests to be acknowledged by Elasticsearch on checkpoints. + * + *

NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT provide + * any strong guarantees for at-least-once delivery of action requests. + */ + public void disableFlushOnCheckpoint() { + this.flushOnCheckpoint = false; + } + + @Override + public void open(Configuration parameters) throws Exception { + client = callBridge.createClient(userConfig); + callBridge.verifyClientConnection(client); + bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); + requestIndexer = + callBridge.createBulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = new BufferingNoOpRequestIndexer(); + elasticsearchSinkFunction.open(); + } + + @Override + public void invoke(T value, Context context) throws Exception { + checkAsyncErrorsAndRequests(); + elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // no initialization needed + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + checkAsyncErrorsAndRequests(); + + if (flushOnCheckpoint) { + while (numPendingRequests.get() != 0) { + bulkProcessor.flush(); + checkAsyncErrorsAndRequests(); + } + } + } + + @Override + public void close() throws Exception { + elasticsearchSinkFunction.close(); + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + client = null; + } + + callBridge.cleanup(); + + // make sure any errors from callbacks are rethrown + checkErrorAndRethrow(); + } + + /** + * Build the {@link BulkProcessor}. + * + *

Note: this is exposed for testing purposes. + */ + @VisibleForTesting + protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { + checkNotNull(listener); + + BulkProcessor.Builder bulkProcessorBuilder = + callBridge.createBulkProcessorBuilder(client, listener); + + // This makes flush() blocking + bulkProcessorBuilder.setConcurrentRequests(0); + + if (bulkProcessorFlushMaxActions != null) { + bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions); + } + + if (bulkProcessorFlushMaxSizeMb != null) { + configureBulkSize(bulkProcessorBuilder); + } + + if (bulkProcessorFlushIntervalMillis != null) { + configureFlushInterval(bulkProcessorBuilder); + } + + // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null + callBridge.configureBulkProcessorBackoff( + bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); + + return bulkProcessorBuilder.build(); + } + + private void configureBulkSize(BulkProcessor.Builder bulkProcessorBuilder) { + final ByteSizeUnit sizeUnit; + if (bulkProcessorFlushMaxSizeMb == -1) { + // bulk size can be disabled with -1, however the ByteSizeValue constructor accepts -1 + // only with BYTES as the size unit + sizeUnit = ByteSizeUnit.BYTES; + } else { + sizeUnit = ByteSizeUnit.MB; + } + bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, sizeUnit)); + } + + private void configureFlushInterval(BulkProcessor.Builder bulkProcessorBuilder) { + if (bulkProcessorFlushIntervalMillis == -1) { + bulkProcessorBuilder.setFlushInterval(null); + } else { + callBridge.configureBulkProcessorFlushInterval( + bulkProcessorBuilder, bulkProcessorFlushIntervalMillis); + } + } + + private void checkErrorAndRethrow() { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in ElasticsearchSink.", cause); + } + } + + private void checkAsyncErrorsAndRequests() { + checkErrorAndRethrow(); + failureRequestIndexer.processBufferedRequests(requestIndexer); + } + + private class BulkProcessorListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) {} + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + if (response.hasFailures()) { + BulkItemResponse itemResponse; + Throwable failure; + RestStatus restStatus; + DocWriteRequest actionRequest; + + try { + for (int i = 0; i < response.getItems().length; i++) { + itemResponse = response.getItems()[i]; + failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); + if (failure != null) { + restStatus = itemResponse.getFailure().getStatus(); + actionRequest = request.requests().get(i); + if (restStatus == null) { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + -1, + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } else { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + restStatus.getStatus(), + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + try { + for (DocWriteRequest writeRequest : request.requests()) { + if (writeRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) writeRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + } + + @VisibleForTesting + long getNumPendingRequests() { + if (flushOnCheckpoint) { + return numPendingRequests.get(); + } else { + throw new UnsupportedOperationException( + "The number of pending requests is not maintained when flushing on checkpoint is disabled."); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java new file mode 100644 index 00000000..0cf85239 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -0,0 +1,83 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RuntimeContext; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream. + * + *

This is used by sinks to prepare elements for sending them to Elasticsearch. + * + *

Example: + * + *

{@code
+ * 				private static class TestElasticSearchSinkFunction implements
+ * 					ElasticsearchSinkFunction> {
+ *
+ * 				public IndexRequest createIndexRequest(Tuple2 element) {
+ * 					Map json = new HashMap<>();
+ * 					json.put("data", element.f1);
+ *
+ * 					return Requests.indexRequest()
+ * 						.index("my-index")
+ * 						.type("my-type")
+ * 						.id(element.f0.toString())
+ * 						.source(json);
+ * 					}
+ *
+ * 			public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
+ * 				indexer.add(createIndexRequest(element));
+ * 			}
+ * 	}
+ *
+ * }
+ * + * @param The type of the element handled by this {@code ElasticsearchSinkFunction} + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface ElasticsearchSinkFunction extends Serializable, Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param ctx runtime context containing information about the sink instance + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void process(T element, RuntimeContext ctx, RequestIndexer indexer); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java new file mode 100644 index 00000000..747c690a --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -0,0 +1,84 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +/** + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Elasticsearch cluster. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface RequestIndexer { + + /** + * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param actionRequests The multiple {@link ActionRequest} to add. + * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} + */ + @Deprecated + default void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + if (actionRequest instanceof IndexRequest) { + add((IndexRequest) actionRequest); + } else if (actionRequest instanceof DeleteRequest) { + add((DeleteRequest) actionRequest); + } else if (actionRequest instanceof UpdateRequest) { + add((UpdateRequest) actionRequest); + } else { + throw new IllegalArgumentException( + "RequestIndexer only supports Index, Delete and Update requests"); + } + } + } + + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000..6c22cf3b --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.time.format.DateTimeFormatter; + +/** Abstract class for time related {@link IndexGenerator}. */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java new file mode 100644 index 00000000..04c76333 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -0,0 +1,169 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.InstantiationUtil; + +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** Accessor methods to elasticsearch options. */ +@Internal +class ElasticsearchConfiguration { + protected final ReadableConfig config; + private final ClassLoader classLoader; + + ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { + this.config = config; + this.classLoader = classLoader; + } + + public ActionRequestFailureHandler getFailureHandler() { + final ActionRequestFailureHandler failureHandler; + String value = config.get(FAILURE_HANDLER_OPTION); + switch (value.toUpperCase()) { + case "FAIL": + failureHandler = new NoOpFailureHandler(); + break; + case "IGNORE": + failureHandler = new IgnoringFailureHandler(); + break; + case "RETRY-REJECTED": + failureHandler = new RetryRejectedExecutionFailureHandler(); + break; + default: + try { + Class failureHandlerClass = Class.forName(value, false, classLoader); + failureHandler = + (ActionRequestFailureHandler) + InstantiationUtil.instantiate(failureHandlerClass); + } catch (ClassNotFoundException e) { + throw new ValidationException( + "Could not instantiate the failure handler class: " + value, e); + } + break; + } + return failureHandler; + } + + public String getDocumentType() { + return config.get(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION); + } + + public int getBulkFlushMaxActions() { + int maxActions = config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxActions == 0 ? -1 : maxActions; + } + + public long getBulkFlushMaxByteSize() { + long maxSize = + config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxSize == 0 ? -1 : maxSize; + } + + public long getBulkFlushInterval() { + long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return interval == 0 ? -1 : interval; + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public boolean isBulkFlushBackoffEnabled() { + return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) + != ElasticsearchConnectorOptions.BackOffType.DISABLED; + } + + public Optional getBulkFlushBackoffType() { + switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { + case CONSTANT: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); + case EXPONENTIAL: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + default: + return Optional.empty(); + } + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { + return !config.get(ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION); + } + + public String getIndex() { + return config.get(ElasticsearchConnectorOptions.INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; + return Objects.equals(config, that.config) && Objects.equals(classLoader, that.classLoader); + } + + @Override + public int hashCode() { + return Objects.hash(config, classLoader); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java new file mode 100644 index 00000000..4838b035 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -0,0 +1,170 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; + +/** + * Options for the Elasticsearch connector. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public class ElasticsearchConnectorOptions { + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + + public static final ConfigOption FAILURE_HANDLER_OPTION = + ConfigOptions.key("failure-handler") + .stringType() + .defaultValue("fail") + .withDescription( + Description.builder() + .text( + "Failure handling strategy in case a request to Elasticsearch fails") + .list( + text( + "\"fail\" (throws an exception if a request fails and thus causes a job failure)"), + text( + "\"ignore\" (ignores failures and drops the request)"), + text( + "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation)"), + text( + "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) + .build()); + + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = + ConfigOptions.key("sink.flush-on-checkpoint") + .booleanType() + .defaultValue(true) + .withDescription("Disables flushing on checkpoint"); + + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(BackOffType.class) + .defaultValue(BackOffType.DISABLED) + .withDescription("Backoff strategy"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_PATH_PREFIX = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code + * DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } + + private ElasticsearchConnectorOptions() {} +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java new file mode 100644 index 00000000..6452d006 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.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.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** Utility methods for validating Elasticsearch properties. */ +@Internal +class ElasticsearchValidationUtils { + + private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + } + + /** + * Checks that the table does not have primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(TableSchema schema) { + schema.getPrimaryKey() + .ifPresent( + key -> { + List illegalTypes = + key.getColumns().stream() + .map( + fieldName -> { + LogicalType logicalType = + schema.getFieldDataType(fieldName) + .get() + .getLogicalType(); + if (logicalType.is( + LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) + .collect(Collectors.toList()); + + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.\n" + + " Elasticsearch sink does not support primary keys on columns of types: %s.", + illegalTypes, ILLEGAL_PRIMARY_KEY_TYPES)); + } + }); + } + + private ElasticsearchValidationUtils() {} +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java new file mode 100644 index 00000000..636f3409 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -0,0 +1,39 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** This interface is responsible to generate index name from given {@link Row} record. */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} + + /** Generate index name according the the given row. */ + String generate(RowData row); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java new file mode 100644 index 00000000..adfcaa46 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.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.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** Base class for {@link IndexGenerator}. */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java new file mode 100644 index 00000000..48f0107b --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -0,0 +1,312 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + return createIndexGenerator(index, schema, ZoneId.systemDefault()); + } + + public static IndexGenerator createIndexGenerator( + String index, TableSchema schema, ZoneId localTimeZoneId) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + schema.getFieldNames(), + schema.getFieldDataTypes(), + indexHelper, + localTimeZoneId); + } else { + return new StaticIndexGenerator(index); + } + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper, + ZoneId localTimeZoneId) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) { + final String dateTimeFormat = + indexHelper.extractDateFormat( + index, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + return indexPrefix + .concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter)) + .concat(indexSuffix); + } + }; + } + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final Pattern dynamicIndexSystemTimeExtractPattern = + Pattern.compile( + ".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } + + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } + + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** Check generate dynamic index is from system time or not. */ + boolean checkIsDynamicIndexWithSystemTimeFormat(String index) { + return dynamicIndexSystemTimeExtractPattern.matcher(index).matches(); + } + + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java new file mode 100644 index 00000000..ae7c522b --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -0,0 +1,130 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** An extractor for a Elasticsearch key from a {@link RowData}. */ +@Internal +class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + private static class ColumnWithIndex { + public TableColumn column; + public int index; + + public ColumnWithIndex(TableColumn column, int index) { + this.column = column; + this.index = index; + } + + public LogicalType getType() { + return column.getType().getLogicalType(); + } + + public int getIndex() { + return index; + } + } + + public static Function createKeyExtractor( + TableSchema schema, String keyDelimiter) { + return schema.getPrimaryKey() + .map( + key -> { + Map namesToColumns = new HashMap<>(); + List tableColumns = schema.getTableColumns(); + for (int i = 0; i < schema.getFieldCount(); i++) { + TableColumn column = tableColumns.get(i); + namesToColumns.put( + column.getName(), new ColumnWithIndex(column, i)); + } + + FieldFormatter[] fieldFormatters = + key.getColumns().stream() + .map(namesToColumns::get) + .map( + column -> + toFormatter( + column.index, column.getType())) + .toArray(FieldFormatter[]::new); + + return (Function) + new KeyExtractor(fieldFormatters, keyDelimiter); + }) + .orElseGet(() -> (Function & Serializable) (row) -> null); + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java new file mode 100644 index 00000000..f5b24180 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java @@ -0,0 +1,54 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.Serializable; + +/** For version-agnostic creating of {@link ActionRequest}s. */ +@Internal +interface RequestFactory extends Serializable { + /** + * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest(String index, String docType, String key); +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java new file mode 100644 index 00000000..7fb1e4d4 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -0,0 +1,140 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.function.Function; + +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ +@Internal +class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1L; + + private final IndexGenerator indexGenerator; + private final String docType; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final Function createKey; + + public RowElasticsearchSinkFunction( + IndexGenerator indexGenerator, + @Nullable String docType, // this is deprecated in es 7+ + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + Function createKey) { + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = docType; + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.createKey = Preconditions.checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + requestFactory.createUpdateRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + requestFactory.createIndexRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = + requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key); + indexer.add(deleteRequest); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) + && Objects.equals(docType, that.docType) + && Objects.equals(serializationSchema, that.serializationSchema) + && contentType == that.contentType + && Objects.equals(requestFactory, that.requestFactory) + && Objects.equals(createKey, that.createKey); + } + + @Override + public int hashCode() { + return Objects.hash( + indexGenerator, + docType, + serializationSchema, + contentType, + requestFactory, + createKey); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java new file mode 100644 index 00000000..1ffcac48 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.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.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** A static {@link IndexGenerator} which generate fixed index name. */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java new file mode 100644 index 00000000..058d4c76 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java @@ -0,0 +1,38 @@ +/* + * 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.elasticsearch.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; + +/** Ignores all kinds of failures and drops the affected {@link ActionRequest}. */ +@Internal +public class IgnoringFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 1662846593501L; + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // ignore failure + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java new file mode 100644 index 00000000..cd6e5063 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -0,0 +1,54 @@ +/* + * 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.elasticsearch.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */ +@Internal +public class NoOpFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 737941343410827885L; + + private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class); + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + // simply fail the sink + throw failure; + } + + @Override + public boolean equals(Object o) { + return o instanceof NoOpFailureHandler; + } + + @Override + public int hashCode() { + return NoOpFailureHandler.class.hashCode(); + } +} diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java new file mode 100644 index 00000000..bfa7b8ca --- /dev/null +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -0,0 +1,59 @@ +/* + * 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.elasticsearch.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.ExceptionUtils; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary {@link + * EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), + * and fails for all other failures. + * + * @deprecated This hase been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = -7423562912824511906L; + + private static final Logger LOG = + LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class); + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { + indexer.add(action); + } else { + // rethrow all other failures + throw failure; + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..64b8573d --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java new file mode 100644 index 00000000..7b856c2d --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.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.elasticsearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; + +import org.slf4j.Logger; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +/** Collection of utility methods for Elasticsearch tests. */ +@Internal +public class ElasticsearchUtil { + + private ElasticsearchUtil() {} + + /** + * Creates a preconfigured {@link ElasticsearchContainer} with limited memory allocation and + * aligns the internal Elasticsearch log levels with the ones used by the capturing logger. + * + * @param dockerImageVersion describing the Elasticsearch image + * @param log to derive the log level from + * @return configured Elasticsearch container + */ + public static ElasticsearchContainer createElasticsearchContainer( + String dockerImageVersion, Logger log) { + String logLevel; + if (log.isTraceEnabled()) { + logLevel = "TRACE"; + } else if (log.isDebugEnabled()) { + logLevel = "DEBUG"; + } else if (log.isInfoEnabled()) { + logLevel = "INFO"; + } else if (log.isWarnEnabled()) { + logLevel = "WARN"; + } else if (log.isErrorEnabled()) { + logLevel = "ERROR"; + } else { + logLevel = "OFF"; + } + + return new ElasticsearchContainer(DockerImageName.parse(dockerImageVersion)) + .withEnv("ES_JAVA_OPTS", "-Xms2g -Xmx2g") + .withEnv("logger.org.elasticsearch", logLevel) + .withLogConsumer(new Slf4jLogConsumer(log)); + } + + /** A mock {@link DynamicTableSink.Context} for Elasticsearch tests. */ + public static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java new file mode 100644 index 00000000..af87158b --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java @@ -0,0 +1,223 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +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.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.function.BiFunction; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchSink}. */ +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchSinkBaseITCase { + protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkBaseITCase.class); + protected static final String ELASTICSEARCH_PASSWORD = "test-password"; + protected static final String ELASTICSEARCH_USER = "elastic"; + + private static boolean failed; + + private RestHighLevelClient client; + private TestClientBase context; + + abstract String getElasticsearchHttpHostAddress(); + + abstract TestClientBase createTestClient(RestHighLevelClient client); + + abstract ElasticsearchSinkBuilderBase< + Tuple2, ? extends ElasticsearchSinkBuilderBase> + getSinkBuilder(); + + private RestHighLevelClient createRestHighLevelClient() { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials(ELASTICSEARCH_USER, ELASTICSEARCH_PASSWORD)); + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(getElasticsearchHttpHostAddress())) + .setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider( + credentialsProvider))); + } + + @BeforeEach + void setUp() { + failed = false; + client = createRestHighLevelClient(); + context = createTestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @ParameterizedTest + @EnumSource(DeliveryGuarantee.class) + void testWriteToElasticSearchWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) + throws Exception { + final String index = "test-es-with-delivery-" + deliveryGuarantee; + boolean failure = false; + try { + runTest(index, false, TestEmitter::jsonEmitter, deliveryGuarantee, null); + } catch (IllegalStateException e) { + failure = true; + assertSame(deliveryGuarantee, DeliveryGuarantee.EXACTLY_ONCE); + } finally { + assertEquals(failure, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); + } + } + + @ParameterizedTest + @MethodSource("elasticsearchEmitters") + void testWriteJsonToElasticsearch( + BiFunction>> + emitterProvider) + throws Exception { + final String index = "test-elasticsearch-sink-" + UUID.randomUUID(); + runTest(index, false, emitterProvider, null); + } + + @Test + void testRecovery() throws Exception { + final String index = "test-recovery-elasticsearch-sink"; + runTest(index, true, TestEmitter::jsonEmitter, new FailingMapper()); + assertTrue(failed); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> + emitterProvider, + @Nullable MapFunction additionalMapper) + throws Exception { + runTest( + index, + allowRestarts, + emitterProvider, + DeliveryGuarantee.AT_LEAST_ONCE, + additionalMapper); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> + emitterProvider, + DeliveryGuarantee deliveryGuarantee, + @Nullable MapFunction additionalMapper) + throws Exception { + final ElasticsearchSink> sink = + getSinkBuilder() + .setHosts(HttpHost.create(getElasticsearchHttpHostAddress())) + .setEmitter(emitterProvider.apply(index, context.getDataFieldName())) + .setBulkFlushMaxActions(5) + .setConnectionUsername(ELASTICSEARCH_USER) + .setConnectionPassword(ELASTICSEARCH_PASSWORD) + .setDeliveryGuarantee(deliveryGuarantee) + .build(); + + final StreamExecutionEnvironment env = new LocalStreamEnvironment(); + env.enableCheckpointing(100L); + if (!allowRestarts) { + env.setRestartStrategy(RestartStrategies.noRestart()); + } + DataStream stream = env.fromSequence(1, 5); + + if (additionalMapper != null) { + stream = stream.map(additionalMapper); + } + + stream.map( + new MapFunction>() { + @Override + public Tuple2 map(Long value) throws Exception { + return Tuple2.of( + value.intValue(), + TestClientBase.buildMessage(value.intValue())); + } + }) + .sinkTo(sink); + env.execute(); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + } + + private static List>>> + elasticsearchEmitters() { + return Lists.newArrayList(TestEmitter::jsonEmitter, TestEmitter::smileEmitter); + } + + private static class FailingMapper implements MapFunction, CheckpointListener { + + private int emittedRecords = 0; + + @Override + public Long map(Long value) throws Exception { + Thread.sleep(50); + emittedRecords++; + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (failed || emittedRecords == 0) { + return; + } + failed = true; + throw new Exception("Expected failure"); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java new file mode 100644 index 00000000..4d1890fe --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -0,0 +1,103 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.DynamicTest; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestFactory; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Tests for {@link ElasticsearchSinkBuilderBase}. */ +@ExtendWith(TestLoggerExtension.class) +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +abstract class ElasticsearchSinkBuilderBaseTest> { + + @TestFactory + Stream testValidBuilders() { + Stream validBuilders = + Stream.of( + createMinimalBuilder(), + createMinimalBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), + createMinimalBuilder() + .setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), + createMinimalBuilder() + .setConnectionUsername("username") + .setConnectionPassword("password")); + + return DynamicTest.stream( + validBuilders, + ElasticsearchSinkBuilderBase::toString, + builder -> assertDoesNotThrow(builder::build)); + } + + @Test + void testDefaultDeliveryGuarantee() { + assertThat(createMinimalBuilder().build().getDeliveryGuarantee()) + .isEqualTo(DeliveryGuarantee.AT_LEAST_ONCE); + } + + @Test + void testThrowIfExactlyOnceConfigured() { + assertThrows( + IllegalStateException.class, + () -> createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); + } + + @Test + void testThrowIfHostsNotSet() { + assertThrows( + NullPointerException.class, + () -> createEmptyBuilder().setEmitter((element, indexer, context) -> {}).build()); + } + + @Test + void testThrowIfEmitterNotSet() { + assertThrows( + NullPointerException.class, + () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build()); + } + + @Test + void testThrowIfSetInvalidTimeouts() { + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionRequestTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setSocketTimeout(-1).build()); + } + + abstract B createEmptyBuilder(); + + abstract B createMinimalBuilder(); +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java new file mode 100644 index 00000000..3db12a40 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -0,0 +1,425 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +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.metrics.MetricNames; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.buildMessage; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchWriter}. */ +@Testcontainers +@ExtendWith(TestLoggerExtension.class) +class ElasticsearchWriterITCase { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriterITCase.class); + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); + + private RestHighLevelClient client; + private TestClientBase context; + private MetricListener metricListener; + + @BeforeEach + void setUp() { + metricListener = new MetricListener(); + client = + new RestHighLevelClient( + RestClient.builder(HttpHost.create(ES_CONTAINER.getHttpHostAddress()))); + context = new TestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @Test + void testWriteOnBulkFlush() throws Exception { + final String index = "test-bulk-flush-without-checkpoint"; + final int flushAfterNActions = 5; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + + // Ignore flush on checkpoint + writer.flush(false); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3, 4); + + // Trigger flush + writer.write(Tuple2.of(5, "test-5"), null); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + + writer.write(Tuple2.of(6, "test-6"), null); + context.assertThatIdsAreNotWritten(index, 6); + + // Force flush + writer.blockingFlushAllActions(); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5, 6); + } + } + + @Test + void testWriteOnBulkIntervalFlush() throws Exception { + final String index = "test-bulk-flush-with-interval"; + + // Configure bulk processor to flush every 1s; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + writer.blockingFlushAllActions(); + } + + context.assertThatIdsAreWritten(index, 1, 2, 3, 4); + } + + @Test + void testWriteOnCheckpoint() throws Exception { + final String index = "test-bulk-flush-with-checkpoint"; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0); + + // Enable flush on checkpoint + try (final ElasticsearchWriter> writer = + createWriter(index, true, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3); + + // Trigger flush + writer.flush(false); + + context.assertThatIdsAreWritten(index, 1, 2, 3); + } + } + + @Test + void testIncrementByteOutMetric() throws Exception { + final String index = "test-inc-byte-out"; + final OperatorIOMetricGroup operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); + final InternalSinkWriterMetricGroup metricGroup = + InternalSinkWriterMetricGroup.mock( + metricListener.getMetricGroup(), operatorIOMetricGroup); + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig, metricGroup)) { + final Counter numBytesOut = operatorIOMetricGroup.getNumBytesOutCounter(); + assertEquals(numBytesOut.getCount(), 0); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + long first = numBytesOut.getCount(); + + assertTrue(first > 0); + + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + assertTrue(numBytesOut.getCount() > first); + } + } + + @Test + void testIncrementRecordsSendMetric() throws Exception { + final String index = "test-inc-records-send"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional recordsSend = + metricListener.getCounter(MetricNames.NUM_RECORDS_SEND); + writer.write(Tuple2.of(1, buildMessage(1)), null); + // Update existing index + writer.write(Tuple2.of(1, "u" + buildMessage(2)), null); + // Delete index + writer.write(Tuple2.of(1, "d" + buildMessage(3)), null); + + writer.blockingFlushAllActions(); + + assertTrue(recordsSend.isPresent()); + assertEquals(recordsSend.get().getCount(), 3L); + } + } + + @Test + void testCurrentSendTime() throws Exception { + final String index = "test-current-send-time"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional> currentSendTime = + metricListener.getGauge("currentSendTime"); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + writer.blockingFlushAllActions(); + + assertTrue(currentSendTime.isPresent()); + assertThat(currentSendTime.get().getValue(), greaterThan(0L)); + } + } + + private ElasticsearchWriter> createWriter( + String index, boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig) { + return createWriter( + index, + flushOnCheckpoint, + bulkProcessorConfig, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + } + + private ElasticsearchWriter> createWriter( + String index, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + SinkWriterMetricGroup metricGroup) { + return new ElasticsearchWriter<>( + Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), + new UpdatingEmitter(index, context.getDataFieldName()), + flushOnCheckpoint, + bulkProcessorConfig, + new TestBulkProcessorBuilderFactory(), + new NetworkClientConfig(null, null, null, null, null, null), + metricGroup, + new TestMailbox()); + } + + private static class TestBulkProcessorBuilderFactory implements BulkProcessorBuilderFactory { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda + // because then deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + } + + private static class UpdatingEmitter implements ElasticsearchEmitter> { + + private final String dataFieldName; + private final String index; + + UpdatingEmitter(String index, String dataFieldName) { + this.index = index; + this.dataFieldName = dataFieldName; + } + + @Override + public void emit( + Tuple2 element, + SinkWriter.Context context, + RequestIndexer indexer) { + + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + + final char action = element.f1.charAt(0); + final String id = element.f0.toString(); + switch (action) { + case 'd': + { + indexer.add(new DeleteRequest(index).id(id)); + break; + } + case 'u': + { + indexer.add(new UpdateRequest().index(index).id(id).doc(document)); + break; + } + default: + { + indexer.add( + new IndexRequest(index) + .id(id) + .type(DOCUMENT_TYPE) + .source(document)); + } + } + } + } + + private static class TestClient extends TestClientBase { + + TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT); + } + } + + private static class TestMailbox implements MailboxExecutor { + + @Override + public void execute( + ThrowingRunnable command, + String descriptionFormat, + Object... descriptionArgs) { + try { + command.run(); + } catch (Exception e) { + throw new RuntimeException("Unexpected error", e); + } + } + + @Override + public void yield() throws InterruptedException, FlinkRuntimeException { + Thread.sleep(100); + } + + @Override + public boolean tryYield() throws FlinkRuntimeException { + return false; + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java new file mode 100644 index 00000000..5d2f6f53 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java @@ -0,0 +1,72 @@ +/* + * 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.elasticsearch.sink; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +abstract class TestClientBase { + + static final String DOCUMENT_TYPE = "test-document-type"; + private static final String DATA_FIELD_NAME = "data"; + final RestHighLevelClient client; + + TestClientBase(RestHighLevelClient client) { + this.client = client; + } + + abstract GetResponse getResponse(String index, int id) throws IOException; + + void assertThatIdsAreNotWritten(String index, int... ids) throws IOException { + for (final int id : ids) { + try { + final GetResponse response = getResponse(index, id); + assertFalse( + response.isExists(), String.format("Id %s is unexpectedly present.", id)); + } catch (ElasticsearchStatusException e) { + assertEquals(404, e.status().getStatus()); + } + } + } + + void assertThatIdsAreWritten(String index, int... ids) + throws IOException, InterruptedException { + for (final int id : ids) { + GetResponse response; + do { + response = getResponse(index, id); + Thread.sleep(10); + } while (response.isSourceEmpty()); + assertEquals(buildMessage(id), response.getSource().get(DATA_FIELD_NAME)); + } + } + + String getDataFieldName() { + return DATA_FIELD_NAME; + } + + static String buildMessage(int id) { + return "test-" + id; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java new file mode 100644 index 00000000..2cfc5eb4 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.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.elasticsearch.sink; + +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; + +class TestEmitter implements ElasticsearchEmitter> { + + private final String index; + private final XContentBuilderProvider xContentBuilderProvider; + private final String dataFieldName; + + public static TestEmitter jsonEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::jsonBuilder); + } + + public static TestEmitter smileEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::smileBuilder); + } + + private TestEmitter( + String index, String dataFieldName, XContentBuilderProvider xContentBuilderProvider) { + this.dataFieldName = dataFieldName; + this.index = index; + this.xContentBuilderProvider = xContentBuilderProvider; + } + + @Override + public void emit( + Tuple2 element, SinkWriter.Context context, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + + private IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + try { + return new IndexRequest(index) + .id(element.f0.toString()) + .type(DOCUMENT_TYPE) + .source(xContentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java new file mode 100644 index 00000000..75dd9b98 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -0,0 +1,358 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +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.table.data.TimestampData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.TestLoggerExtension; + +import org.apache.http.HttpHost; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.search.SearchHits; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.io.IOException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.table.api.Expressions.row; + +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchDynamicSinkBaseITCase { + + abstract String getElasticsearchHttpHostAddress(); + + abstract ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory(); + + abstract Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException; + + abstract SearchHits makeSearchRequest(RestHighLevelClient client, String index) + throws IOException; + + abstract long getTotalSearchHits(SearchHits searchHits); + + abstract TestContext getPrefilledTestContext(String index); + + abstract String getConnectorSql(String index); + + private RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(getElasticsearchHttpHostAddress()))); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + ElasticsearchDynamicSinkFactoryBase sinkFactory = getDynamicSinkFactory(); + + DynamicTableSink.SinkRuntimeProvider runtimeProvider = + sinkFactory + .createDynamicTableSink( + getPrefilledTestContext(index).withSchema(schema).build()) + .getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); + + final SinkV2Provider sinkProvider = (SinkV2Provider) runtimeProvider; + final Sink sink = sinkProvider.createSink(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).sinkTo(sink); + environment.execute(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, index, "1_2012-12-12T12:12:12"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, index, "1_2012-12-12T12:12:12"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = makeSearchRequest(client, index); + if (getTotalSearchHits(hits) < 2) { + Thread.sleep(200); + } + } while (getTotalSearchHits(hits) < 2 && deadline.hasTimeLeft()); + + if (getTotalSearchHits(hits) < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + Assertions.assertEquals(resultSet, expectedSet); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, "dynamic-index-2012-12-12", "1"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsWithDynamicIndexFromSystemTime() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + tableEnvironment.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "Asia/Shanghai"); + + String dynamicIndex1 = + "dynamic-index-" + + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai"))) + + "_index"; + String index = "dynamic-index-{now()|yyyy-MM-dd}_index"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + String dynamicIndex2 = + "dynamic-index-" + + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai"))) + + "_index"; + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + Map response; + try { + response = makeGetRequest(client, dynamicIndex1, "1"); + } catch (ElasticsearchStatusException e) { + if (e.status() == RestStatus.NOT_FOUND) { + response = makeGetRequest(client, dynamicIndex2, "1"); + } else { + throw e; + } + } + + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java new file mode 100644 index 00000000..8fa0a498 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -0,0 +1,274 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.function.Executable; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for validation in {@link ElasticsearchDynamicSinkFactoryBase}. */ +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchDynamicSinkFactoryBaseTest { + + abstract ElasticsearchDynamicSinkFactoryBase createSinkFactory(); + + abstract TestContext createPrefilledTestContext(); + + void assertValidationException(String expectedMessage, Executable executable) { + ValidationException thrown = Assertions.assertThrows(ValidationException.class, executable); + Assertions.assertEquals(expectedMessage, thrown.getMessage()); + } + + @Test + public void validateWrongIndex() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "'index' must not be empty", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION.key(), + "") + .build())); + } + + @Test + public void validateWrongHosts() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "wrong-host") + .build())); + } + + @Test + public void validateWrongFlushSize() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_MAX_SIZE_OPTION + .key(), + "1kb") + .build())); + } + + @Test + public void validateWrongRetries() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build())); + } + + @Test + public void validateWrongMaxActions() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.max-actions' must be at least 1. Got: -2", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_MAX_ACTIONS_OPTION + .key(), + "-2") + .build())); + } + + @Test + public void validateWrongBackoffDelay() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "Invalid value for option 'sink.bulk-flush.backoff.delay'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_BACKOFF_DELAY_OPTION + .key(), + "-1s") + .build())); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical( + "a", + DataTypes.BIGINT() + .notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes + .BIGINT(), + DataTypes + .STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes + .FIELD( + "a", + DataTypes + .BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer + .INSTANCE) + .notNull()), + Column.physical( + "g", + DataTypes.BYTES() + .notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList( + "a", "b", "c", "d", "e", + "f", "g")))) + .build())); + } + + @Test + public void validateWrongCredential() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.USERNAME_OPTION.key(), + "username") + .withOption( + ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), + "") + .build())); + } + + @Test + public void validateDynamicIndexOnChangelogStream() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + DynamicTableSink sink = + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION.key(), + "dynamic-index-{now()|yyyy-MM-dd}_index") + .build()); + + ChangelogMode changelogMode = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.DELETE) + .addContainedKind(RowKind.INSERT) + .build(); + assertValidationException( + "Dynamic indexing based on system time only works on append only stream.", + () -> sink.getChangelogMode(changelogMode)); + } + + @Test + public void testSinkParallelism() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + DynamicTableSink sink = + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption(SINK_PARALLELISM.key(), "2") + .build()); + assertThat(sink).isInstanceOf(ElasticsearchDynamicSink.class); + ElasticsearchDynamicSink esSink = (ElasticsearchDynamicSink) sink; + SinkV2Provider provider = + (SinkV2Provider) esSink.getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); + assertThat(2).isEqualTo(provider.getParallelism().get()); + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java new file mode 100644 index 00000000..7840bfa8 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java @@ -0,0 +1,333 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +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.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.Arrays; +import java.util.List; + +/** Suite tests for {@link IndexGenerator}. */ +public class IndexGeneratorTest { + + private static final List fieldNames = + Arrays.asList( + "id", + "item", + "log_ts", + "log_date", + "order_timestamp", + "log_time", + "local_datetime", + "local_date", + "local_time", + "note", + "status"); + + private static final List dataTypes = + Arrays.asList( + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.DATE().bridgedTo(Date.class), + DataTypes.TIMESTAMP().bridgedTo(Timestamp.class), + DataTypes.TIME().bridgedTo(Time.class), + DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class), + DataTypes.DATE().bridgedTo(LocalDate.class), + DataTypes.TIME().bridgedTo(LocalTime.class), + DataTypes.STRING(), + DataTypes.BOOLEAN()); + + private static final List rows = + Arrays.asList( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) Date.valueOf("2020-03-18").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-18 12:12:14")), + (int) + (Time.valueOf("12:12:14").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)), + (int) LocalDate.of(2020, 3, 18).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test1", + true), + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:22:14").getTime(), + (int) Date.valueOf("2020-03-19").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-19 12:22:21")), + (int) + (Time.valueOf("12:22:21").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)), + (int) LocalDate.of(2020, 3, 19).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test2", + false)); + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDateTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromSystemTime() { + List supportedUseCases = + Arrays.asList( + "now()", + "NOW()", + "now( )", + "NOW(\t)", + "\t NOW( ) \t", + "current_timestamp", + "CURRENT_TIMESTAMP", + "\tcurrent_timestamp\t", + " current_timestamp "); + + supportedUseCases.stream() + .forEach( + f -> { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + fieldNames, + dataTypes); + indexGenerator.open(); + // The date may change during the running of the unit test. + // Generate expected index-name based on the current time + // before and after calling the generate method. + String expectedIndex1 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + String actualIndex = indexGenerator.generate(rows.get(1)); + String expectedIndex2 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + Assertions.assertTrue( + actualIndex.equals(expectedIndex1) + || actualIndex.equals(expectedIndex2)); + }); + + List invalidUseCases = + Arrays.asList( + "now", + "now(", + "NOW", + "NOW)", + "current_timestamp()", + "CURRENT_TIMESTAMP()", + "CURRENT_timestamp"); + invalidUseCases.stream() + .forEach( + f -> { + String expectedExceptionMsg = + String.format( + "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}'," + + " please check the field name.", + f, f); + try { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + fieldNames, + dataTypes); + indexGenerator.open(); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + }); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{unknown_ts|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{id|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd HH:mm}", fieldNames, dataTypes); + } catch (UnsupportedTemporalTypeException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", fieldNames, dataTypes); + } catch (IllegalArgumentException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java new file mode 100644 index 00000000..e2110ca7 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java @@ -0,0 +1,147 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +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.table.data.TimestampData; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** Tests for {@link KeyExtractor}. */ +public class KeyExtractorTest { + @Test + public void testSimpleKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + Assertions.assertEquals(key, "12"); + } + + @Test + public void testNoPrimaryKey() { + List logicalTypesWithIndex = Collections.emptyList(); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + Assertions.assertEquals(key, null); + } + + @Test + public void testTwoFieldsKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.TIMESTAMP().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + Assertions.assertEquals(key, "12_2012-12-12T12:12:12"); + } + + @Test + public void testAllTypesKey() { + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.TINYINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 1, DataTypes.SMALLINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.INT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 3, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 4, DataTypes.BOOLEAN().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 5, DataTypes.FLOAT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 6, DataTypes.DOUBLE().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 7, DataTypes.STRING().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 8, DataTypes.TIMESTAMP().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 9, + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE() + .notNull() + .getLogicalType()), + new LogicalTypeWithIndex( + 10, DataTypes.TIME().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 11, DataTypes.DATE().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + Assertions.assertEquals( + key, + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15"); + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java new file mode 100644 index 00000000..0d7bc551 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java @@ -0,0 +1,72 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ +class TestContext { + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(ResolvedSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + Collections.emptyMap(), + new Configuration(), + TestContext.class.getClassLoader(), + false); + } + + public TestContext withOption(String key, String value) { + options.put(key, value); + return this; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java new file mode 100644 index 00000000..98c40930 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -0,0 +1,742 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.MultiShotLatch; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.Requests; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Suite of tests for {@link ElasticsearchSinkBase}. */ +public class ElasticsearchSinkBaseTest { + + /** Verifies that the collection given to the sink is not modified. */ + @Test + public void testCollectionArgumentNotModified() { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, "true"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, "CONSTANT"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, "1"); + + new DummyElasticsearchSink<>( + Collections.unmodifiableMap(userConfig), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + } + + /** + * Tests that any item failure in the listener callbacks is rethrown on an immediately following + * invoke call. + */ + @Test + public void testItemFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the failure + Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any item failure in the listener callbacks is rethrown on an immediately following + * checkpoint. + */ + @Test + public void testItemFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the failure + Assert.assertTrue( + e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any item failure in the listener callbacks due to flushing on an immediately + * following checkpoint is rethrown; we set a timeout because the test will not finish if the + * logic is broken. + */ + @Test(timeout = 5000) + public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + + List mockResponsesList = new ArrayList<>(2); + mockResponsesList.add(null); // the first request in a bulk will succeed + mockResponsesList.add( + new Exception( + "artificial failure for record")); // the second request in a bulk will fail + sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); + + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request (1 request only, thus should succeed) + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // let the snapshot-triggered flush continue (2 records in the bulk, so the 2nd one should + // fail) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the failure from the 2nd request + Assert.assertTrue( + e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following + * invoke call. + */ + @Test + public void testBulkFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the bulk request failure + Assert.assertTrue( + e.getCause().getMessage().contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following + * checkpoint. + */ + @Test + public void testBulkFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue( + e.getCause() + .getCause() + .getMessage() + .contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks due to flushing on an immediately + * following checkpoint is rethrown; we set a timeout because the test will not finish if the + * logic is broken. + */ + @Test(timeout = 5000) + public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList((Exception) null)); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // for the snapshot-triggered flush, we let the bulk request fail completely + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + + // let the snapshot-triggered flush continue (bulk request should fail completely) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue( + e.getCause() + .getCause() + .getMessage() + .contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that the sink correctly waits for pending requests (including re-added requests) on + * checkpoints; we set a timeout because the test will not finish if the logic is broken. + */ + @Test(timeout = 5000) + public void testAtLeastOnceSink() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new DummyRetryFailureHandler()); // use a failure handler that simply + // re-adds requests + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures; + // it contains 1 request, which will fail and re-added to the next bulk request + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + sink.continueFlush(); + + // since the previous flush should have resulted in a request re-add from the failure + // handler, + // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd + // flush + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // current number of pending request should be 1 due to the re-add + Assert.assertEquals(1, sink.getNumPendingRequests()); + + // this time, let the bulk request succeed, so no-more requests are re-added + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList((Exception) null)); + + sink.continueFlush(); + + // the snapshot should finish with no exceptions + snapshotThread.sync(); + + testHarness.close(); + } + + /** + * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is + * disabled, the snapshot method does indeed finishes without waiting for pending requests; we + * set a timeout because the test will not finish if the logic is broken. + */ + @Test(timeout = 5000) + public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new DummyRetryFailureHandler()); + sink.disableFlushOnCheckpoint(); // disable flushing + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // the snapshot should not block even though we haven't flushed the bulk request + testHarness.snapshot(1L, 1000L); + + testHarness.close(); + } + + @Test + public void testOpenAndCloseInSinkFunction() throws Exception { + SimpleClosableSinkFunction sinkFunction = new SimpleClosableSinkFunction<>(); + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap<>(), sinkFunction, new DummyRetryFailureHandler()); + + sink.open(mock(Configuration.class)); + sink.close(); + + Assert.assertTrue(sinkFunction.openCalled); + Assert.assertTrue(sinkFunction.closeCalled); + } + + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 5051907841570096991L; + + private transient BulkProcessor mockBulkProcessor; + private transient BulkRequest nextBulkRequest = new BulkRequest(); + private transient MultiShotLatch flushLatch = new MultiShotLatch(); + + private List mockItemFailuresList; + private Throwable nextBulkFailure; + + public DummyElasticsearchSink( + Map userConfig, + ElasticsearchSinkFunction sinkFunction, + ActionRequestFailureHandler failureHandler) { + super(new DummyElasticsearchApiCallBridge(), userConfig, sinkFunction, failureHandler); + } + + /** + * This method is used to mimic a scheduled bulk request; we need to do this manually + * because we are mocking the BulkProcessor. + */ + public void manualBulkRequestWithAllPendingRequests() { + flushLatch.trigger(); // let the flush + mockBulkProcessor.flush(); + } + + /** + * On non-manual flushes, i.e. when flush is called in the snapshot method implementation, + * usages need to explicitly call this to allow the flush to continue. This is useful to + * make sure that specific requests get added to the next bulk request for flushing. + */ + public void continueFlush() { + flushLatch.trigger(); + } + + /** + * Set the list of mock failures to use for the next bulk of item responses. A {@code null} + * means that the response is successful, failed otherwise. + * + *

The list is used with corresponding order to the requests in the bulk, i.e. the first + * request uses the response at index 0, the second requests uses the response at index 1, + * etc. + */ + public void setMockItemFailuresListForNextBulkItemResponses( + List mockItemFailuresList) { + this.mockItemFailuresList = mockItemFailuresList; + } + + /** + * Let the next bulk request fail completely with the provided throwable. If this is set, + * the failures list provided with setMockItemFailuresListForNextBulkItemResponses is not + * respected. + */ + public void setFailNextBulkRequestCompletely(Throwable failure) { + this.nextBulkFailure = failure; + } + + public BulkProcessor getMockBulkProcessor() { + return mockBulkProcessor; + } + + /** + * Override the bulk processor build process to provide a mock implementation, but reuse the + * listener implementation in our mock to test that the listener logic works correctly with + * request flushing logic. + */ + @Override + protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { + this.mockBulkProcessor = mock(BulkProcessor.class); + + when(mockBulkProcessor.add(any(IndexRequest.class))) + .thenAnswer( + new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) + throws Throwable { + // intercept the request and add it to our mock bulk request + nextBulkRequest.add( + (IndexRequest) invocationOnMock.getArgument(0)); + + return null; + } + }); + + doAnswer( + new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) + throws Throwable { + while (nextBulkRequest.numberOfActions() > 0) { + // wait until we are allowed to continue with the flushing + flushLatch.await(); + + // create a copy of the accumulated mock requests, so that + // re-added requests from the failure handler are included + // in the next bulk + BulkRequest currentBulkRequest = nextBulkRequest; + nextBulkRequest = new BulkRequest(); + + listener.beforeBulk(123L, currentBulkRequest); + + if (nextBulkFailure == null) { + BulkItemResponse[] mockResponses = + new BulkItemResponse + [currentBulkRequest.requests().size()]; + for (int i = 0; + i < currentBulkRequest.requests().size(); + i++) { + Exception mockItemFailure = + mockItemFailuresList.get(i); + + if (mockItemFailure == null) { + // the mock response for the item is success + mockResponses[i] = + new BulkItemResponse( + i, + DocWriteRequest.OpType.INDEX, + mock(DocWriteResponse.class)); + } else { + // the mock response for the item is failure + mockResponses[i] = + new BulkItemResponse( + i, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure( + "index", + "type", + "id", + mockItemFailure)); + } + } + + listener.afterBulk( + 123L, + currentBulkRequest, + new BulkResponse(mockResponses, 1000L)); + } else { + listener.afterBulk( + 123L, currentBulkRequest, nextBulkFailure); + } + } + + return null; + } + }) + .when(mockBulkProcessor) + .flush(); + + return mockBulkProcessor; + } + } + + private static class DummyElasticsearchApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -4272760730959041699L; + + @Override + public Client createClient(Map clientConfig) { + return mock(Client.class); + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + Client client, BulkProcessor.Listener listener) { + return null; + } + + @Nullable + @Override + public Throwable extractFailureCauseFromBulkItemResponse( + BulkItemResponse bulkItemResponse) { + if (bulkItemResponse.isFailed()) { + return new Exception(bulkItemResponse.getFailure().getMessage()); + } else { + return null; + } + } + + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + // no need for this in the test cases here + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + // no need for this in the test cases here + } + + @Override + public void verifyClientConnection(Client client) { + // no need for this in the test cases here + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new TestRequestIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + } + + private static class SimpleSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = -176739293659135148L; + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + Map json = new HashMap<>(); + json.put("data", element); + + indexer.add(Requests.indexRequest().index("index").type("type").id("id").source(json)); + } + } + + private static class SimpleClosableSinkFunction + implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1872065917794006848L; + + private boolean openCalled; + private boolean closeCalled; + + @Override + public void open() { + openCalled = true; + } + + @Override + public void close() { + closeCalled = true; + } + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {} + } + + private static class DummyRetryFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 5400023700099200745L; + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + indexer.add(action); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java new file mode 100644 index 00000000..84342fc6 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -0,0 +1,203 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; +import org.apache.flink.test.util.AbstractTestBase; + +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.junit.Assert.fail; + +/** + * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} + * implementations. + * + * @param Elasticsearch client type + * @param The address type to use + */ +public abstract class ElasticsearchSinkTestBase + extends AbstractTestBase { + + protected abstract RestHighLevelClient getClient(); + + protected abstract String getClusterName(); + + /** Tests that the Elasticsearch sink works properly with json. */ + public void runElasticsearchSinkTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with cbor. */ + public void runElasticsearchSinkCborTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-cbor-index", SourceSinkDataTestKit::getCborSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with smile. */ + public void runElasticsearchSinkSmileTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with yaml. */ + public void runElasticsearchSinkYamlTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-yaml-index", SourceSinkDataTestKit::getYamlSinkFunction); + } + + private void runElasticSearchSinkTest( + String index, + Function>> functionFactory) + throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createElasticsearchSinkForEmbeddedNode( + 1, getClusterName(), functionFactory.apply(index))); + + env.execute("Elasticsearch Sink Test"); + + // verify the results + RestHighLevelClient client = getClient(); + + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code + * null}. + */ + public void runNullAddressesTest() { + try { + createElasticsearchSink( + 1, getClusterName(), null, SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException | NullPointerException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. + */ + public void runEmptyAddressesTest() { + try { + createElasticsearchSink( + 1, + getClusterName(), + Collections.emptyList(), + SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** Tests whether the Elasticsearch sink fails when there is no cluster to connect to. */ + public void runInvalidElasticsearchClusterTest() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createElasticsearchSinkForNode( + 1, + "invalid-cluster-name", + SourceSinkDataTestKit.getJsonSinkFunction("test"), + "123.123.123.123")); // incorrect ip address + + try { + env.execute("Elasticsearch Sink Test"); + } catch (JobExecutionException expectedException) { + // every ES version throws a different exception in case of timeouts, so don't bother + // asserting on the exception + // test passes + return; + } + + fail(); + } + + /** Utility method to create a user config map. */ + protected Map createUserConfig(int bulkFlushMaxActions, String clusterName) { + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", clusterName); + userConfig.put( + ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, + String.valueOf(bulkFlushMaxActions)); + userConfig.put("transport.tcp.connect_timeout", "5s"); + + return userConfig; + } + + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ + protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List addresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction); + + /** + * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch + * node. + * + *

This case is singled out from {@link + * ElasticsearchSinkTestBase#createElasticsearchSink(int, String, List, + * ElasticsearchSinkFunction)} because the Elasticsearch Java API to do so is incompatible + * across different versions. + */ + protected abstract ElasticsearchSinkBase, C> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) + throws Exception; + + /** + * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. + */ + protected abstract ElasticsearchSinkBase, C> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) + throws Exception; +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java new file mode 100644 index 00000000..738ddd60 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -0,0 +1,54 @@ +/* + * 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.elasticsearch; + +import org.elasticsearch.client.Client; + +import java.io.File; + +/** + * The {@link EmbeddedElasticsearchNodeEnvironment} is used in integration tests to manage + * Elasticsearch embedded nodes. + * + *

NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific + * implementations for the tests, concrete implementations must be named {@code + * EmbeddedElasticsearchNodeEnvironmentImpl}. It must also be located under the same package. The + * intentional package-private accessibility of this interface enforces that. + */ +public interface EmbeddedElasticsearchNodeEnvironment { + + /** + * Start an embedded Elasticsearch node instance. Calling this method multiple times + * consecutively should not restart the embedded node. + * + * @param tmpDataFolder The temporary data folder for the embedded node to use. + * @param clusterName The name of the cluster that the embedded node should be configured with. + */ + void start(File tmpDataFolder, String clusterName) throws Exception; + + /** Close the embedded node, if previously started. */ + void close() throws Exception; + + /** + * Returns a client to communicate with the embedded node. + * + * @return Client to communicate with the embedded node. Returns {@code null} if the embedded + * node wasn't started or is closed. + */ + Client getClient(); +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java new file mode 100644 index 00000000..dc0c146e --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.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.streaming.connectors.elasticsearch; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + */ +class TestRequestIndexer implements RequestIndexer { + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + TestRequestIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java new file mode 100644 index 00000000..a5f77595 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java @@ -0,0 +1,282 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +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.table.data.TimestampData; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Tests for {@link IndexGeneratorFactory}. */ +public class IndexGeneratorFactoryTest extends TestLogger { + + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + schema = + new TableSchema.Builder() + .field("id", DataTypes.INT()) + .field("item", DataTypes.STRING()) + .field("log_ts", DataTypes.BIGINT()) + .field("log_date", DataTypes.DATE()) + .field("log_time", DataTypes.TIME()) + .field("order_timestamp", DataTypes.TIMESTAMP()) + .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("status", DataTypes.BOOLEAN()) + .build(); + + rows = new ArrayList<>(); + rows.add( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-18").toEpochDay(), + (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), + TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), + true)); + rows.add( + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-19").toEpochDay(), + (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), + TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromSystemTime() { + List supportedUseCases = + Arrays.asList( + "now()", + "NOW()", + "now( )", + "NOW(\t)", + "\t NOW( ) \t", + "current_timestamp", + "CURRENT_TIMESTAMP", + "\tcurrent_timestamp\t", + " current_timestamp "); + + supportedUseCases.stream() + .forEach( + f -> { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), schema); + indexGenerator.open(); + // The date may change during the running of the unit test. + // Generate expected index-name based on the current time + // before and after calling the generate method. + String expectedIndex1 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + String actualIndex = indexGenerator.generate(rows.get(1)); + String expectedIndex2 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + Assert.assertTrue( + actualIndex.equals(expectedIndex1) + || actualIndex.equals(expectedIndex2)); + }); + + List invalidUseCases = + Arrays.asList( + "now", + "now(", + "NOW", + "NOW)", + "current_timestamp()", + "CURRENT_TIMESTAMP()", + "CURRENT_timestamp"); + invalidUseCases.stream() + .forEach( + f -> { + String expectedExceptionMsg = + String.format( + "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}'," + + " please check the field name.", + f, f); + try { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + schema); + indexGenerator.open(); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + }); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java new file mode 100644 index 00000000..bcfb68da --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java @@ -0,0 +1,135 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +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.table.data.TimestampData; + +import org.junit.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; + +/** Tests for {@link KeyExtractor}. */ +public class KeyExtractorTest { + @Test + public void testSimpleKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .primaryKey("a") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, equalTo("12")); + } + + @Test + public void testNoPrimaryKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, nullValue()); + } + + @Test + public void testTwoFieldsKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "c") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + assertThat(key, equalTo("12_2012-12-12T12:12:12")); + } + + @Test + public void testAllTypesKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.TINYINT().notNull()) + .field("b", DataTypes.SMALLINT().notNull()) + .field("c", DataTypes.INT().notNull()) + .field("d", DataTypes.BIGINT().notNull()) + .field("e", DataTypes.BOOLEAN().notNull()) + .field("f", DataTypes.FLOAT().notNull()) + .field("g", DataTypes.DOUBLE().notNull()) + .field("h", DataTypes.STRING().notNull()) + .field("i", DataTypes.TIMESTAMP().notNull()) + .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) + .field("k", DataTypes.TIME().notNull()) + .field("l", DataTypes.DATE().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + assertThat( + key, + equalTo( + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java new file mode 100644 index 00000000..40405142 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -0,0 +1,72 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ +class TestContext { + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(ResolvedSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + Collections.emptyMap(), + new Configuration(), + TestContext.class.getClassLoader(), + false); + } + + public TestContext withOption(String key, String value) { + options.put(key, value); + return this; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java new file mode 100644 index 00000000..cd11b70d --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java @@ -0,0 +1,107 @@ +/* + * 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.elasticsearch.testutils; + +import org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironment; +import org.apache.flink.util.InstantiationUtil; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.client.AdminClient; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.ClusterAdminClient; +import org.elasticsearch.common.unit.TimeValue; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertThat; + +/** A resource that starts an embedded elasticsearch cluster. */ +public class ElasticsearchResource extends ExternalResource { + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchResource.class); + private EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; + private final TemporaryFolder tempFolder = new TemporaryFolder(); + + private final String clusterName; + + public ElasticsearchResource(String clusterName) { + this.clusterName = clusterName; + } + + @Override + protected void before() throws Throwable { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + // dynamically load version-specific implementation of the Elasticsearch embedded node + // environment + Class clazz = + Class.forName( + "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); + embeddedNodeEnv = + (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); + + tempFolder.create(); + embeddedNodeEnv.start(tempFolder.newFolder(), clusterName); + + waitForCluster(); + } + + /** Blocks until the cluster is ready and data nodes/nodes are live. */ + private void waitForCluster() { + AdminClient adminClient = embeddedNodeEnv.getClient().admin(); + ClusterAdminClient clusterAdminClient = adminClient.cluster(); + + ClusterHealthRequestBuilder requestBuilder = clusterAdminClient.prepareHealth("_all"); + requestBuilder = requestBuilder.setTimeout(TimeValue.timeValueSeconds(120)); + + ActionFuture healthFuture = + clusterAdminClient.health(requestBuilder.request()); + + ClusterHealthResponse health = healthFuture.actionGet(TimeValue.timeValueSeconds(120)); + + assertThat(health.getNumberOfNodes(), greaterThanOrEqualTo(1)); + assertThat(health.getNumberOfDataNodes(), greaterThanOrEqualTo(1)); + } + + @Override + protected void after() { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shutting down embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + try { + embeddedNodeEnv.close(); + tempFolder.delete(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public Client getClient() { + return embeddedNodeEnv.getClient(); + } +} diff --git a/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java new file mode 100644 index 00000000..584ed4d3 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -0,0 +1,155 @@ +/* + * 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.elasticsearch.testutils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.junit.Assert; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** + * This class contains utilities and a pre-defined source function and Elasticsearch Sink function + * used to simulate and verify data used in tests. + */ +public class SourceSinkDataTestKit { + + private static final int NUM_ELEMENTS = 20; + + private static final String DATA_PREFIX = "message #"; + private static final String DATA_FIELD_NAME = "data"; + + private static final String TYPE_NAME = "flink-es-test-type"; + + /** + * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 - + * 20. + */ + public static class TestDataSourceFunction implements SourceFunction> { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceFunction.SourceContext> ctx) + throws Exception { + for (int i = 0; i < NUM_ELEMENTS && running; i++) { + ctx.collect(Tuple2.of(i, DATA_PREFIX + i)); + } + } + + @Override + public void cancel() { + running = false; + } + } + + public static ElasticsearchSinkFunction> getCborSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::cborBuilder); + } + + public static ElasticsearchSinkFunction> getJsonSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::jsonBuilder); + } + + public static ElasticsearchSinkFunction> getSmileSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::smileBuilder); + } + + public static ElasticsearchSinkFunction> getYamlSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::yamlBuilder); + } + + private static class TestElasticsearchSinkFunction + implements ElasticsearchSinkFunction> { + private static final long serialVersionUID = 1L; + + private final String index; + private final XContentBuilderProvider contentBuilderProvider; + + /** + * Create the sink function, specifying a target Elasticsearch index. + * + * @param index Name of the target Elasticsearch index. + */ + public TestElasticsearchSinkFunction( + String index, XContentBuilderProvider contentBuilderProvider) { + this.index = index; + this.contentBuilderProvider = contentBuilderProvider; + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(DATA_FIELD_NAME, element.f1); + + try { + return new IndexRequest(index, TYPE_NAME, element.f0.toString()) + .source(contentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void process( + Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + } + + /** + * Verify the results in an Elasticsearch index. The results must first be produced into the + * index using a {@link TestElasticsearchSinkFunction}; + * + * @param client The client to use to connect to Elasticsearch + * @param index The index to check + */ + public static void verifyProducedSinkData(RestHighLevelClient client, String index) + throws IOException { + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = + client.get( + new GetRequest(index, TYPE_NAME, Integer.toString(i)), + RequestOptions.DEFAULT); + Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } +} diff --git a/flink-connector-elasticsearch-base/src/test/resources/archunit.properties b/flink-connector-elasticsearch-base/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations diff --git a/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000..835c2ec9 --- /dev/null +++ b/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 b/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 new file mode 100644 index 00000000..e69de29b diff --git a/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d b/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d new file mode 100644 index 00000000..341409ac --- /dev/null +++ b/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d @@ -0,0 +1,12 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase 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 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.elasticsearch.table.Elasticsearch6DynamicSinkITCase 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 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 diff --git a/flink-connector-elasticsearch6/archunit-violations/stored.rules b/flink-connector-elasticsearch6/archunit-violations/stored.rules new file mode 100644 index 00000000..10c711f0 --- /dev/null +++ b/flink-connector-elasticsearch6/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Tue Feb 22 12:17:08 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=25e52d29-fa7e-42fa-a571-b5c76235df52 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=db3972e4-f3a3-45b2-9643-27cba0cef09d diff --git a/flink-connector-elasticsearch6/pom.xml b/flink-connector-elasticsearch6/pom.xml new file mode 100644 index 00000000..3adae87a --- /dev/null +++ b/flink-connector-elasticsearch6/pom.xml @@ -0,0 +1,185 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + .. + + + flink-connector-elasticsearch6 + Flink : Connectors : Elasticsearch 6 + + jar + + + + 6.8.20 + + + + + + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + true + + + + + + org.apache.flink + flink-connector-elasticsearch-base + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + + + + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + + + org.testcontainers + elasticsearch + test + + + + org.apache.flink + flink-test-utils + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base + ${project.version} + + + org.elasticsearch + elasticsearch + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + + + test-jar + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + org.apache.logging.log4j + log4j-api + provided + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + + org.apache.flink + flink-json + ${flink.version} + test + + + + + + org.apache.flink + flink-architecture-tests-test + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + + + + + diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java new file mode 100644 index 00000000..c90ccaca --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -0,0 +1,131 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; + +/** + * Builder to construct an Elasticsearch 6 compatible {@link ElasticsearchSink}. + * + *

The following example shows the minimal setup to create a ElasticsearchSink that submits + * actions on checkpoint or the default number of actions was buffered (1000). + * + *

{@code
+ * ElasticsearchSink sink = new Elasticsearch6SinkBuilder()
+ *     .setHosts(new HttpHost("localhost:9200")
+ *     .setEmitter((element, context, indexer) -> {
+ *          indexer.add(
+ *              new IndexRequest("my-index","my-type")
+ *              .id(element.f0.toString())
+ *              .source(element.f1)
+ *          );
+ *      })
+ *     .build();
+ * }
+ * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class Elasticsearch6SinkBuilder + extends ElasticsearchSinkBuilderBase> { + + public Elasticsearch6SinkBuilder() {} + + @Override + public Elasticsearch6SinkBuilder setEmitter( + ElasticsearchEmitter emitter) { + super.setEmitter(emitter); + return self(); + } + + @Override + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { + return new BulkProcessorBuilderFactory() { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a + // lambda because then + // deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener + bulkResponseActionListener) { + client.bulkAsync(bulkRequest, bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval( + new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = + BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + }; + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java new file mode 100644 index 00000000..03705509 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java @@ -0,0 +1,37 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; + +import static org.apache.flink.connector.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; + +/** Elasticsearch 6 specific configuration. */ +@Internal +final class Elasticsearch6Configuration extends ElasticsearchConfiguration { + + Elasticsearch6Configuration(ReadableConfig config) { + super(config); + } + + public String getDocumentType() { + return config.get(DOCUMENT_TYPE_OPTION); + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java new file mode 100644 index 00000000..d9ff53ca --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java @@ -0,0 +1,39 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Options specific for the Elasticsearch 6 connector. Public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ +@PublicEvolving +public final class Elasticsearch6ConnectorOptions extends ElasticsearchConnectorOptions { + + private Elasticsearch6ConnectorOptions() {} + + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java new file mode 100644 index 00000000..69576978 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nullable; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link ElasticsearchDynamicSink}. */ +@Internal +public class Elasticsearch6DynamicSinkFactory extends ElasticsearchDynamicSinkFactoryBase { + private static final String FACTORY_IDENTIFIER = "elasticsearch-6"; + + public Elasticsearch6DynamicSinkFactory() { + super(FACTORY_IDENTIFIER, Elasticsearch6SinkBuilder::new); + } + + @Override + ElasticsearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) { + return new Elasticsearch6Configuration(helper.getOptions()); + } + + @Nullable + @Override + String getDocumentType(ElasticsearchConfiguration configuration) { + return ((Elasticsearch6Configuration) configuration).getDocumentType(); + } + + @Override + void validateConfiguration(ElasticsearchConfiguration config) { + super.validateConfiguration(config); + Elasticsearch6Configuration configuration = (Elasticsearch6Configuration) config; + validate( + !StringUtils.isNullOrWhitespaceOnly(configuration.getDocumentType()), + () -> String.format("'%s' must not be empty", DOCUMENT_TYPE_OPTION.key())); + } + + @Override + public Set> requiredOptions() { + Set> requiredOptions = super.requiredOptions(); + requiredOptions.add(DOCUMENT_TYPE_OPTION); + return requiredOptions; + } + + @Override + public Set> forwardOptions() { + return Stream.concat(super.forwardOptions().stream(), Stream.of(DOCUMENT_TYPE_OPTION)) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java new file mode 100644 index 00000000..8b833216 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.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.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 6 specific configuration. */ +@Internal +final class Elasticsearch6Configuration extends ElasticsearchConfiguration { + Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch6Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+     *     connector.hosts = http://host_name:9092;http://host_name:9093
+     * 
+ */ + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java new file mode 100644 index 00000000..1a2cdd18 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -0,0 +1,340 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + * + * @deprecated Please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder} to build a {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} instead. + */ +@Deprecated +@PublicEvolving +final class Elasticsearch6DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch6Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + config.getDocumentType(), + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch6"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, username, password); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch6RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index, docType, key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java new file mode 100644 index 00000000..73f5cf88 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -0,0 +1,184 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ +@Internal +public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION, DOCUMENT_TYPE_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch6Configuration config = + new Elasticsearch6Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch6DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-6"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java new file mode 100644 index 00000000..bd5d5a6a --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -0,0 +1,148 @@ +/* + * 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.elasticsearch6; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ +@Internal +public class Elasticsearch6ApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6ApiCallBridge.class); + + /** User-provided HTTP Host. */ + private final List httpHosts; + + /** The factory to configure the rest client. */ + private final RestClientFactory restClientFactory; + + Elasticsearch6ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) { + RestClientBuilder builder = + RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client::bulkAsync, listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + builder.setFlushInterval(TimeValue.timeValueMillis(flushIntervalMillis)); + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = + BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = + BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch6BulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping()) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java new file mode 100644 index 00000000..dfeb54ad --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.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.streaming.connectors.elasticsearch6; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + *

Note: This class is binary compatible to Elasticsearch 6. + */ +@Internal +class Elasticsearch6BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + Elasticsearch6BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java new file mode 100644 index 00000000..22b5051d --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -0,0 +1,269 @@ +/* + * 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.elasticsearch6; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Elasticsearch 6.x sink that requests multiple {@link ActionRequest ActionRequests} against a + * cluster for each incoming element. + * + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch + * cluster. The sink will fail if no cluster can be connected to using the provided transport + * addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest + * ActionRequests}. This will buffer elements before sending a request to the cluster. The behaviour + * of the {@code BulkProcessor} can be configured using these config keys: + * + *

    + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
+ * + *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation + * of {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + * @deprecated This sink has been deprecated in favor of {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} + */ +@Deprecated +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super( + new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), + bulkRequestsConfig, + elasticsearchSinkFunction, + failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + * @deprecated This has been deprecated, please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder}. + */ + @Deprecated + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link + * RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} + * from the incoming element. + */ + public Builder( + List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>( + bulkRequestsConfig, + httpHosts, + elasticsearchSinkFunction, + failureHandler, + restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) + && Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) + && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) + && Objects.equals(failureHandler, builder.failureHandler) + && Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } +} diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java new file mode 100644 index 00000000..d11eb970 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.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.streaming.connectors.elasticsearch6; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} + * internally used in the {@link ElasticsearchSink}. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); +} diff --git a/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..29a85938 --- /dev/null +++ b/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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.streaming.connectors.elasticsearch.table.Elasticsearch6DynamicSinkFactory diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..4f99f9ee --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,57 @@ +/* + * 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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.core.importer.Location; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +import java.util.regex.Pattern; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch6" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + TestCodeArchitectureTest.IncludeES6ImportOption.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); + + /** Only include ES6 related locations. */ + public static final class IncludeES6ImportOption implements ImportOption { + private static final Pattern ES6 = Pattern.compile(".*elasticsearch6.*"); + + @Override + public boolean includes(Location location) { + return location.matches(ES6); + } + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java new file mode 100644 index 00000000..3de02f50 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java @@ -0,0 +1,37 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.http.HttpHost; + +/** Tests for {@link Elasticsearch6SinkBuilder}. */ +class Elasticsearch6SinkBuilderTest + extends ElasticsearchSinkBuilderBaseTest> { + + @Override + Elasticsearch6SinkBuilder createEmptyBuilder() { + return new Elasticsearch6SinkBuilder<>(); + } + + @Override + Elasticsearch6SinkBuilder createMinimalBuilder() { + return new Elasticsearch6SinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java new file mode 100644 index 00000000..c947cda9 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.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.elasticsearch.sink; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.client.RestHighLevelClient; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class Elasticsearch6SinkITCase extends ElasticsearchSinkBaseITCase { + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + ElasticsearchUtil.createElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_6, LOG) + .withPassword(ELASTICSEARCH_PASSWORD); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + TestClientBase createTestClient(RestHighLevelClient client) { + return new Elasticsearch6TestClient(client); + } + + @Override + Elasticsearch6SinkBuilder> getSinkBuilder() { + return new Elasticsearch6SinkBuilder<>(); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java new file mode 100644 index 00000000..a29eb03a --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java @@ -0,0 +1,36 @@ +/* + * 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.elasticsearch.sink; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +class Elasticsearch6TestClient extends TestClientBase { + + Elasticsearch6TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, DOCUMENT_TYPE, Integer.toString(id))); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java new file mode 100644 index 00000000..d8053d4c --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -0,0 +1,55 @@ +/* + * 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.elasticsearch.table; + +import org.junit.jupiter.api.Test; + +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ +public class Elasticsearch6DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { + @Override + ElasticsearchDynamicSinkFactoryBase createSinkFactory() { + return new Elasticsearch6DynamicSinkFactory(); + } + + @Override + TestContext createPrefilledTestContext() { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345"); + } + + @Test + public void validateEmptyConfiguration() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index", + () -> sinkFactory.createDynamicTableSink(context().build())); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java new file mode 100644 index 00000000..055a822a --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -0,0 +1,103 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.search.SearchHits; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.io.IOException; +import java.util.Map; + +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; + +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@Testcontainers +public class Elasticsearch6DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { + + private static final Logger LOG = + LoggerFactory.getLogger(Elasticsearch6DynamicSinkITCase.class); + + private static final String DOCUMENT_TYPE = "MyType"; + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_6, LOG); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory() { + return new Elasticsearch6DynamicSinkFactory(); + } + + @Override + Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException { + return client.get(new GetRequest(index, DOCUMENT_TYPE, id)).getSource(); + } + + @Override + SearchHits makeSearchRequest(RestHighLevelClient client, String index) throws IOException { + return client.search(new SearchRequest(index)).getHits(); + } + + @Override + long getTotalSearchHits(SearchHits hits) { + return hits.getTotalHits(); + } + + @Override + TestContext getPrefilledTestContext(String index) { + return context() + .withOption(Elasticsearch6ConnectorOptions.INDEX_OPTION.key(), index) + .withOption( + Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOCUMENT_TYPE) + .withOption( + Elasticsearch6ConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } + + @Override + String getConnectorSql(String index) { + return String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", Elasticsearch6ConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOCUMENT_TYPE) + + String.format( + "'%s'='%s'\n", + Elasticsearch6ConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java new file mode 100644 index 00000000..e99abbe9 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -0,0 +1,250 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ +public class Elasticsearch6DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("document-type", "MyType") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "MyIndex") + .withOption("document-type", "MyType") + .withOption("hosts", "wrong-host") + .build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java new file mode 100644 index 00000000..6717e8fa --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -0,0 +1,400 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +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.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.search.SearchHits; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch6DynamicSink}. */ +public class Elasticsearch6DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); + + @SuppressWarnings("deprecation") + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + String myType = "MyType"; + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions + .DOCUMENT_TYPE_OPTION + .key(), + myType) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + RestHighLevelClient client = getClient(); + Map response = + client.get( + new GetRequest(index, myType, "1_2012-12-12T12:12:12"), + RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = + client.get( + new GetRequest(index, myType, "1_2012-12-12T12:12:12"), + RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); + if (hits.getTotalHits() < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits() < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = + client.get( + new GetRequest("dynamic-index-2012-12-12", myType, "1"), + RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java new file mode 100644 index 00000000..f8ab0ab0 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch6DynamicSink} parameters. */ +public class Elasticsearch6DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java new file mode 100644 index 00000000..c2fd9f8d --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -0,0 +1,133 @@ +/* + * 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.elasticsearch6; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.util.DockerImageVersions; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.elasticsearch.ElasticsearchContainer; + +import java.util.ArrayList; +import java.util.List; + +/** IT cases for the {@link ElasticsearchSink}. */ +public class ElasticsearchSinkITCase + extends ElasticsearchSinkTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_6, LOG); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); + } + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + + @Test + public void testNullAddresses() { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception { + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, + clusterName, + elasticsearchSinkFunction, + elasticsearchContainer.getHttpHostAddress()); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String hostAddress) { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(HttpHost.create(hostAddress)); + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } +} diff --git a/flink-connector-elasticsearch6/src/test/resources/archunit.properties b/flink-connector-elasticsearch6/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations diff --git a/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000..835c2ec9 --- /dev/null +++ b/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f b/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f new file mode 100644 index 00000000..e69de29b diff --git a/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e b/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e new file mode 100644 index 00000000..e6b90b93 --- /dev/null +++ b/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e @@ -0,0 +1,12 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase 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 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.elasticsearch.table.Elasticsearch7DynamicSinkITCase 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 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 diff --git a/flink-connector-elasticsearch7/archunit-violations/stored.rules b/flink-connector-elasticsearch7/archunit-violations/stored.rules new file mode 100644 index 00000000..f2bb6a71 --- /dev/null +++ b/flink-connector-elasticsearch7/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Tue Feb 22 12:17:16 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=1af7baaa-05dc-452a-9de7-653c8b3b324f +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=e1f30f33-c61c-4707-8c78-a3a80479564e diff --git a/flink-connector-elasticsearch7/pom.xml b/flink-connector-elasticsearch7/pom.xml new file mode 100644 index 00000000..90c8427b --- /dev/null +++ b/flink-connector-elasticsearch7/pom.xml @@ -0,0 +1,181 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + + + flink-connector-elasticsearch7 + Flink : Connectors : Elasticsearch 7 + + jar + + + + 7.10.2 + + + + + + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + + + org.apache.flink + flink-connector-elasticsearch-base + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + provided + true + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + org.apache.httpcomponents + httpcore-nio + + + + + + + org.apache.httpcomponents + httpcore-nio + 4.4.12 + + + + + + org.testcontainers + elasticsearch + test + + + + org.apache.flink + flink-test-utils + test + + + + org.apache.flink + flink-streaming-java + ${flink.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + org.apache.flink + flink-connector-test-utils + ${flink.version} + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + test + + + + + org.apache.flink + flink-json + ${flink.version} + test + + + + + + org.apache.flink + flink-architecture-tests-test + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + + + + + diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java new file mode 100644 index 00000000..567f7740 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -0,0 +1,133 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; + +/** + * Builder to construct an Elasticsearch 7 compatible {@link ElasticsearchSink}. + * + *

The following example shows the minimal setup to create a ElasticsearchSink that submits + * actions on checkpoint or the default number of actions was buffered (1000). + * + *

{@code
+ * ElasticsearchSink sink = new Elasticsearch7SinkBuilder()
+ *     .setHosts(new HttpHost("localhost:9200")
+ *     .setEmitter((element, context, indexer) -> {
+ *          indexer.add(
+ *              new IndexRequest("my-index")
+ *              .id(element.f0.toString())
+ *              .source(element.f1)
+ *          );
+ *      })
+ *     .build();
+ * }
+ * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class Elasticsearch7SinkBuilder + extends ElasticsearchSinkBuilderBase> { + + public Elasticsearch7SinkBuilder() {} + + @Override + public Elasticsearch7SinkBuilder setEmitter( + ElasticsearchEmitter emitter) { + super.setEmitter(emitter); + return self(); + } + + @Override + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { + return new BulkProcessorBuilderFactory() { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a + // lambda because then + // deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener + bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval( + new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = + BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + }; + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java new file mode 100644 index 00000000..02c8eebd --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -0,0 +1,33 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder; +import org.apache.flink.table.factories.DynamicTableSinkFactory; + +/** A {@link DynamicTableSinkFactory} for discovering {@link ElasticsearchDynamicSink}. */ +@Internal +public class Elasticsearch7DynamicSinkFactory extends ElasticsearchDynamicSinkFactoryBase { + private static final String FACTORY_IDENTIFIER = "elasticsearch-7"; + + public Elasticsearch7DynamicSinkFactory() { + super(FACTORY_IDENTIFIER, Elasticsearch7SinkBuilder::new); + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java new file mode 100644 index 00000000..6bd28cf4 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -0,0 +1,70 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 7 specific configuration. */ +@Internal +final class Elasticsearch7Configuration extends ElasticsearchConfiguration { + Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch7Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java new file mode 100644 index 00000000..1926e445 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -0,0 +1,335 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@Internal +final class Elasticsearch7DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch7Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + null, // this is deprecated in es 7+ + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, password, username); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index).id(key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java new file mode 100644 index 00000000..70b0134d --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -0,0 +1,184 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ +@Internal +public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch7Configuration config = + new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch7DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-7"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java new file mode 100644 index 00000000..b076a30e --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -0,0 +1,152 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. */ +@Internal +public class Elasticsearch7ApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class); + + /** User-provided HTTP Host. */ + private final List httpHosts; + + /** The factory to configure the rest client. */ + private final RestClientFactory restClientFactory; + + Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) { + RestClientBuilder builder = + RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder( + (request, bulkListener) -> + client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), + listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + builder.setFlushInterval(TimeValue.timeValueMillis(flushIntervalMillis)); + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = + BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = + BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch7BulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java new file mode 100644 index 00000000..866c0586 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.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.streaming.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + *

Note: This class is binary compatible to Elasticsearch 7. + */ +@Internal +class Elasticsearch7BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + Elasticsearch7BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java new file mode 100644 index 00000000..4a5de6a1 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -0,0 +1,269 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Elasticsearch 7.x sink that requests multiple {@link ActionRequest ActionRequests} against a + * cluster for each incoming element. + * + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch + * cluster. The sink will fail if no cluster can be connected to using the provided transport + * addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest + * ActionRequests}. This will buffer elements before sending a request to the cluster. The behaviour + * of the {@code BulkProcessor} can be configured using these config keys: + * + *

    + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
+ * + *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation + * of {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + * @deprecated This sink has been deprecated in favor of {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} + */ +@Deprecated +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super( + new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), + bulkRequestsConfig, + elasticsearchSinkFunction, + failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + * @deprecated This has been deprecated, please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder}. + */ + @Deprecated + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link + * RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} + * from the incoming element. + */ + public Builder( + List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>( + bulkRequestsConfig, + httpHosts, + elasticsearchSinkFunction, + failureHandler, + restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) + && Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) + && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) + && Objects.equals(failureHandler, builder.failureHandler) + && Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } +} diff --git a/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java new file mode 100644 index 00000000..9e056712 --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.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.streaming.connectors.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} + * internally used in the {@link ElasticsearchSink}. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); +} diff --git a/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..10e4846e --- /dev/null +++ b/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..53898b90 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,57 @@ +/* + * 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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.core.importer.Location; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +import java.util.regex.Pattern; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch7" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + TestCodeArchitectureTest.IncludeES7ImportOption.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); + + /** Only include ES7 related locations. */ + public static final class IncludeES7ImportOption implements ImportOption { + private static final Pattern ES7 = Pattern.compile(".*elasticsearch7.*"); + + @Override + public boolean includes(Location location) { + return location.matches(ES7); + } + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java new file mode 100644 index 00000000..0d6250ea --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java @@ -0,0 +1,37 @@ +/* + * 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.elasticsearch.sink; + +import org.apache.http.HttpHost; + +/** Tests for {@link Elasticsearch7SinkBuilder}. */ +class Elasticsearch7SinkBuilderTest + extends ElasticsearchSinkBuilderBaseTest> { + + @Override + Elasticsearch7SinkBuilder createEmptyBuilder() { + return new Elasticsearch7SinkBuilder<>(); + } + + @Override + Elasticsearch7SinkBuilder createMinimalBuilder() { + return new Elasticsearch7SinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java new file mode 100644 index 00000000..62ed8dea --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.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.elasticsearch.sink; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.client.RestHighLevelClient; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class Elasticsearch7SinkITCase extends ElasticsearchSinkBaseITCase { + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + ElasticsearchUtil.createElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_7, LOG) + .withPassword(ELASTICSEARCH_PASSWORD); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + TestClientBase createTestClient(RestHighLevelClient client) { + return new Elasticsearch7TestClient(client); + } + + @Override + ElasticsearchSinkBuilderBase, ? extends ElasticsearchSinkBuilderBase> + getSinkBuilder() { + return new Elasticsearch7SinkBuilder<>(); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java new file mode 100644 index 00000000..969c1796 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java @@ -0,0 +1,37 @@ +/* + * 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.elasticsearch.sink; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +class Elasticsearch7TestClient extends TestClientBase { + + Elasticsearch7TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java new file mode 100644 index 00000000..e8247725 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.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.elasticsearch.table; + +import org.junit.jupiter.api.Test; + +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ +public class Elasticsearch7DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { + @Override + ElasticsearchDynamicSinkFactoryBase createSinkFactory() { + return new Elasticsearch7DynamicSinkFactory(); + } + + @Override + TestContext createPrefilledTestContext() { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345"); + } + + @Test + public void validateEmptyConfiguration() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index", + () -> sinkFactory.createDynamicTableSink(context().build())); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java new file mode 100644 index 00000000..caddcda9 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -0,0 +1,97 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.search.SearchHits; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +import java.io.IOException; +import java.util.Map; + +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; + +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@Testcontainers +public class Elasticsearch7DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { + + private static final Logger LOG = + LoggerFactory.getLogger(Elasticsearch7DynamicSinkITCase.class); + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory() { + return new Elasticsearch7DynamicSinkFactory(); + } + + @Override + Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException { + return client.get(new GetRequest(index, id), RequestOptions.DEFAULT).getSource(); + } + + @Override + SearchHits makeSearchRequest(RestHighLevelClient client, String index) throws IOException { + return client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); + } + + @Override + long getTotalSearchHits(SearchHits searchHits) { + return searchHits.getTotalHits().value; + } + + @Override + TestContext getPrefilledTestContext(String index) { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } + + @Override + String getConnectorSql(String index) { + return String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java new file mode 100644 index 00000000..3ff21ed1 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -0,0 +1,234 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ +public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java new file mode 100644 index 00000000..3534f4a8 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -0,0 +1,377 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +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.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.search.SearchHits; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch7DynamicSink}. */ +public class Elasticsearch7DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + + @SuppressWarnings("deprecation") + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + RestHighLevelClient client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL," + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); + if (hits.getTotalHits().value < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits().value < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", "1"), RequestOptions.DEFAULT) + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java new file mode 100644 index 00000000..2928c8db --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * 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.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch7DynamicSink} parameters. */ +public class Elasticsearch7DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java new file mode 100644 index 00000000..e5a5527c --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -0,0 +1,133 @@ +/* + * 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.elasticsearch7; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.util.DockerImageVersions; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.elasticsearch.ElasticsearchContainer; + +import java.util.ArrayList; +import java.util.List; + +/** IT cases for the {@link ElasticsearchSink}. */ +public class ElasticsearchSinkITCase + extends ElasticsearchSinkTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); + } + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + + @Test + public void testNullAddresses() { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception { + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, + clusterName, + elasticsearchSinkFunction, + elasticsearchContainer.getHttpHostAddress()); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String hostAddress) { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(HttpHost.create(hostAddress)); + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } +} diff --git a/flink-connector-elasticsearch7/src/test/resources/archunit.properties b/flink-connector-elasticsearch7/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations diff --git a/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties new file mode 100644 index 00000000..835c2ec9 --- /dev/null +++ b/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-sql-connector-elasticsearch6/pom.xml b/flink-sql-connector-elasticsearch6/pom.xml new file mode 100644 index 00000000..a3f8651b --- /dev/null +++ b/flink-sql-connector-elasticsearch6/pom.xml @@ -0,0 +1,149 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + + + flink-sql-connector-elasticsearch6 + Flink : Connectors : SQL : Elasticsearch 6 + + jar + + + + org.apache.flink + flink-connector-elasticsearch6 + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + *:* + + + + com.carrotsearch:hppc + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + + + + + org.elasticsearch:elasticsearch + + config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + + forbidden/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.elasticsearch.** + META-INF/LICENSE.txt + + + + + + + org.apache.commons + org.apache.flink.elasticsearch6.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch6.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch6.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch6.shaded.org.elasticsearch + + + com.fasterxml.jackson + org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson + + + com.github.mustachejava + org.apache.flink.elasticsearch6.shaded.com.github.mustachejava + + + + + + + + + + diff --git a/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..de1f4f37 --- /dev/null +++ b/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -0,0 +1,47 @@ +flink-sql-connector-elasticsearch6 +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- commons-codec:commons-codec:1.15 +- commons-logging:commons-logging:1.1.3 +- org.apache.httpcomponents:httpasyncclient:4.1.2 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 +- org.apache.httpcomponents:httpcore-nio:4.4.5 +- org.apache.lucene:lucene-analyzers-common:7.7.3 +- org.apache.lucene:lucene-backward-codecs:7.7.3 +- org.apache.lucene:lucene-core:7.7.3 +- org.apache.lucene:lucene-grouping:7.7.3 +- org.apache.lucene:lucene-highlighter:7.7.3 +- org.apache.lucene:lucene-join:7.7.3 +- org.apache.lucene:lucene-memory:7.7.3 +- org.apache.lucene:lucene-misc:7.7.3 +- org.apache.lucene:lucene-queries:7.7.3 +- org.apache.lucene:lucene-queryparser:7.7.3 +- org.apache.lucene:lucene-sandbox:7.7.3 +- org.apache.lucene:lucene-spatial:7.7.3 +- org.apache.lucene:lucene-spatial-extras:7.7.3 +- org.apache.lucene:lucene-spatial3d:7.7.3 +- org.apache.lucene:lucene-suggest:7.7.3 +- org.elasticsearch:elasticsearch:6.8.20 +- org.elasticsearch:elasticsearch-cli:6.8.20 +- org.elasticsearch:elasticsearch-core:6.8.20 +- org.elasticsearch:elasticsearch-secure-sm:6.8.20 +- org.elasticsearch:elasticsearch-x-content:6.8.20 +- org.elasticsearch.client:elasticsearch-rest-client:6.8.20 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:6.8.20 +- org.elasticsearch.plugin:aggs-matrix-stats-client:6.8.20 +- org.elasticsearch.plugin:parent-join-client:6.8.20 +- org.elasticsearch.plugin:rank-eval-client:6.8.20 +- org.elasticsearch.plugin:lang-mustache-client:6.8.20 +- com.github.spullara.mustache.java:compiler:0.9.3 diff --git a/flink-sql-connector-elasticsearch7/pom.xml b/flink-sql-connector-elasticsearch7/pom.xml new file mode 100644 index 00000000..15292f9b --- /dev/null +++ b/flink-sql-connector-elasticsearch7/pom.xml @@ -0,0 +1,158 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + + + flink-sql-connector-elasticsearch7 + Flink : Connectors : SQL : Elasticsearch 7 + + jar + + + + org.apache.flink + flink-connector-elasticsearch7 + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + *:* + + + + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + + + + + org.elasticsearch:elasticsearch + + config/** + modules.txt + plugins.txt + org/joda/** + + org/elasticsearch/bootstrap/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + + forbidden/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.elasticsearch.** + META-INF/LICENSE.txt + + + + + + + org.apache.commons + org.apache.flink.elasticsearch7.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch7.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch7.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch7.shaded.org.elasticsearch + + + com.fasterxml.jackson + org.apache.flink.elasticsearch7.shaded.com.fasterxml.jackson + + + com.carrotsearch.hppc + org.apache.flink.elasticsearch7.shaded.com.carrotsearch.hppc + + + com.github.mustachejava + org.apache.flink.elasticsearch7.shaded.com.github.mustachejava + + + net.jpountz + org.apache.flink.elasticsearch7.shaded.net.jpountz + + + + + + + + + + diff --git a/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..fa04fce1 --- /dev/null +++ b/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -0,0 +1,52 @@ +flink-sql-connector-elasticsearch7 +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.carrotsearch:hppc:0.8.1 +- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- com.github.spullara.mustache.java:compiler:0.9.6 +- commons-codec:commons-codec:1.15 +- commons-logging:commons-logging:1.1.3 +- org.apache.httpcomponents:httpasyncclient:4.1.4 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 +- org.apache.httpcomponents:httpcore-nio:4.4.12 +- org.apache.lucene:lucene-analyzers-common:8.7.0 +- org.apache.lucene:lucene-backward-codecs:8.7.0 +- org.apache.lucene:lucene-core:8.7.0 +- org.apache.lucene:lucene-grouping:8.7.0 +- org.apache.lucene:lucene-highlighter:8.7.0 +- org.apache.lucene:lucene-join:8.7.0 +- org.apache.lucene:lucene-memory:8.7.0 +- org.apache.lucene:lucene-misc:8.7.0 +- org.apache.lucene:lucene-queries:8.7.0 +- org.apache.lucene:lucene-queryparser:8.7.0 +- org.apache.lucene:lucene-sandbox:8.7.0 +- org.apache.lucene:lucene-spatial:8.7.0 +- org.apache.lucene:lucene-spatial-extras:8.7.0 +- org.apache.lucene:lucene-spatial3d:8.7.0 +- org.apache.lucene:lucene-suggest:8.7.0 +- org.elasticsearch:elasticsearch:7.10.2 +- org.elasticsearch:elasticsearch-cli:7.10.2 +- org.elasticsearch:elasticsearch-core:7.10.2 +- org.elasticsearch:elasticsearch-geo:7.10.2 +- org.elasticsearch:elasticsearch-secure-sm:7.10.2 +- org.elasticsearch:elasticsearch-x-content:7.10.2 +- org.elasticsearch:elasticsearch-plugin-classloader:7.10.2 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.10.2 +- org.elasticsearch.client:elasticsearch-rest-client:7.10.2 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.10.2 +- org.elasticsearch.plugin:lang-mustache-client:7.10.2 +- org.elasticsearch.plugin:mapper-extras-client:7.10.2 +- org.elasticsearch.plugin:parent-join-client:7.10.2 +- org.elasticsearch.plugin:rank-eval-client:7.10.2 +- org.lz4:lz4-java:1.8.0 diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..01587eeb --- /dev/null +++ b/pom.xml @@ -0,0 +1,2025 @@ + + + + + + org.apache + apache + 20 + + + 4.0.0 + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + Flink : Connectors : + pom + https://flink.apache.org + 2022 + + + + The Apache Software License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/apache/flink-connector-elasticsearch + git@github.com:apache/flink-connector-elasticsearch.git + + scm:git:https://gitbox.apache.org/repos/asf/flink-connector-elasticsearch.git + + + + + flink-connector-elasticsearch-base + flink-connector-elasticsearch6 + flink-connector-elasticsearch7 + + + + UTF-8 + UTF-8 + + + 1C + + ${flink.forkCount} + true + + 1.15-SNAPSHOT + 15.0 + 2.12.4 + 1.8 + 1.7.32 + 2.17.1 + + ${target.java.version} + ${target.java.version} + 2.1.1 + + 2.12.7 + 2.12 + 0.7.6 + + 4.13.2 + 5.8.1 + 3.21.0 + 0.22.0 + 1.16.2 + + 2.21.0 + 2.0.9 + 1.3 + + 0.10.9.3 + 2.27.0 + 3.17.3 + 3.14.9 + 1.8.0 + + false + 1.14.0 + tools/japicmp-output + + validate + + 2.4.2 + + + + **/*Test.* + + + + + org.apache.flink + flink-shaded-force-shading + ${flink.shaded.version} + + + + + + + org.slf4j + slf4j-api + provided + + + + + com.google.code.findbugs + jsr305 + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + org.junit.vintage + junit-vintage-engine + test + + + + org.assertj + assertj-core + test + + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + + + org.powermock + powermock-module-junit4 + ${powermock.version} + jar + test + + + + org.powermock + powermock-api-mockito2 + ${powermock.version} + jar + test + + + org.mockito + mockito-core + + + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + jar + test + + + + org.testcontainers + junit-jupiter + test + + + + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + org.apache.logging.log4j + log4j-api + test + + + + org.apache.logging.log4j + log4j-core + test + + + + + org.apache.logging.log4j + log4j-1.2-api + test + + + + org.apache.flink + flink-test-utils-junit + test + + + + org.apache.httpcomponents + httpcore-nio + + + + + + + + + + + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + + + org.apache.flink + flink-architecture-tests-base + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-test + ${flink.version} + test + + + + + com.google.code.findbugs + jsr305 + 1.3.9 + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + org.xerial.snappy + snappy-java + 1.1.8.3 + + + + org.lz4 + lz4-java + ${lz4.version} + + + + com.github.oshi + oshi-core + 3.4.0 + + + + + org.apache.avro + avro + ${avro.version} + + + + + org.hamcrest + hamcrest-core + ${hamcrest.version} + + + + + net.bytebuddy + byte-buddy + 1.8.22 + + + + + net.bytebuddy + byte-buddy-agent + 1.8.22 + + + + + org.objenesis + objenesis + 2.1 + + + + + commons-logging + commons-logging + 1.1.3 + + + + com.fasterxml.jackson + jackson-bom + pom + import + 2.13.0 + + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + + com.squareup.okhttp3 + logging-interceptor + ${okhttp.version} + + + + + jakarta.activation + jakarta.activation-api + 1.2.1 + provided + + + + + jakarta.xml.bind + jakarta.xml.bind-api + 2.3.2 + provided + + + + + org.junit + junit-bom + ${junit5.version} + pom + import + + + + junit + junit + ${junit4.version} + + + + org.assertj + assertj-core + ${assertj.version} + test + + + + + commons-cli + commons-cli + 1.5.0 + + + + commons-io + commons-io + 2.11.0 + + + + + commons-collections + commons-collections + 3.2.2 + + + + + commons-configuration + commons-configuration + 1.7 + + + + commons-codec + commons-codec + 1.15 + + + + org.apache.commons + commons-math3 + 3.6.1 + + + + org.apache.commons + commons-compress + 1.21 + + + + + org.javassist + javassist + 3.24.0-GA + + + + + joda-time + joda-time + 2.5 + + + + org.joda + joda-convert + 1.7 + + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + org.scala-lang + scala-library + ${scala.version} + + + + org.scala-lang + scala-reflect + ${scala.version} + + + + org.scala-lang + scala-compiler + ${scala.version} + + + + org.scalatest + scalatest_${scala.binary.version} + 3.0.0 + test + + + + + javax.xml.bind + jaxb-api + ${jaxb.api.version} + + + + + javax.activation + javax.activation-api + ${javax.activation.api.version} + + + + + org.apache.httpcomponents + httpcore + 4.4.14 + + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + + + org.apache.httpcomponents + httpcore-nio + 4.4.12 + + + + org.reflections + reflections + 0.9.10 + test + + + + + com.lmax + disruptor + 3.4.2 + + + + org.yaml + snakeyaml + 1.27 + + + io.netty + netty-bom + 4.1.70.Final + pom + import + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import + + + + com.tngtech.archunit + archunit + ${archunit.version} + test + + + + com.tngtech.archunit + archunit-junit5 + ${archunit.version} + test + + + + + + + + + + + sql-jars + + + !skipSqlJars + + + + flink-sql-connector-elasticsearch6 + flink-sql-connector-elasticsearch7 + + + + + scala-2.12 + + 2.12.7 + 2.12 + + + + !old-and-unsupported-scala-version + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-versions + + enforce + + + + + + *:*_2.11 + *:*_2.10 + + Scala 2.10/2.11 dependencies are not allowed for Scala 2.12 builds. This can be caused by hard-coded scala versions, where the 'scala.binary.version' property should be used instead. + + + + + + + + + + + + enable-adaptive-scheduler + + + + org.apache.maven.plugins + maven-surefire-plugin + + + true + + + + + + + + + java11 + + [11,) + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.7 + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + com.github.siom79.japicmp + japicmp-maven-plugin + + + javax.xml.bind + jaxb-api + 2.3.0 + + + com.sun.xml.bind + jaxb-impl + 2.3.1 + + + com.sun.xml.bind + jaxb-core + 2.3.0 + + + javax.activation + activation + 1.1.1 + + + + + org.apache.maven.plugins + maven-surefire-plugin + + org.apache.flink.testutils.junit.FailsOnJava11 + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + + + + + + + + + + java17 + + [17,) + + + + + + + com.diffplug.spotless + spotless-maven-plugin + + + true + + + + + + + + + java11-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 11 + 11 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + java17-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + fast + + + fast + + + + + + + org.apache.rat + apache-rat-plugin + + true + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + com.diffplug.spotless + spotless-maven-plugin + + true + + + + org.scalastyle + scalastyle-maven-plugin + + true + + + + org.apache.maven.plugins + maven-enforcer-plugin + + true + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + com.github.siom79.japicmp + japicmp-maven-plugin + + true + + + + + + + + + check-convergence + + + check-convergence + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + ${flink.convergence.phase} + + + + + + + + + spotbugs + + + spotbugs + + + + + + com.github.hazendaz.spotbugs + spotbugs-maven-plugin + 3.0.6 + + + + findbugs-run + compile + + check + + + + + + true + Low + default + ${project.build.directory}/spotbugs + ${rootDir}/tools/maven/spotbugs-exclude.xml + true + + + + + org.codehaus.mojo + xml-maven-plugin + 1.0.1 + + + verify + + transform + + + + + + + ${project.build.directory}/spotbugs + ${project.build.directory}/spotbugs + + plain.xsl + + + + .html + + + + + + + + com.github.hazendaz.spotbugs + spotbugs-maven-plugin + 3.0.6 + + + + + + + + + + aggregate-scaladoc + + + + + + docs-and-source + + + docs-and-source + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + attach-javadocs + + jar + + + + + + + + + + release + + + release + + + + 1.8 + + + + + org.apache.maven.plugins + maven-gpg-plugin + 1.4 + + + sign-artifacts + verify + + sign + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + (,3.3) + + + 1.8.0 + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + attach-javadocs + + jar + + + + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.1 + + forked-path + false + ${arguments} -Psonatype-oss-release + + + + + + + + + + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + false + + true + true + + + + + + + org.apache.rat + apache-rat-plugin + 0.13 + false + + + verify + + check + + + + + true + false + 0 + + + + AL2 + Apache License 2.0 + + + Licensed to the Apache Software Foundation (ASF) under one + + + + + + Apache License 2.0 + + + + + **/.*/** + **/*.prefs + **/*.log + + docs/**/jquery* + docs/**/bootstrap* + docs/themes/book/** + docs/**/anchor* + **/resources/**/font-awesome/** + **/resources/**/jquery* + **/resources/**/bootstrap* + docs/resources/** + docs/public/** + docs/assets/github.css + docs/static/flink-header-logo.svg + docs/static/figs/*.svg + docs/static/font-awesome/** + docs/static/flink-header-logo.svg + docs/static/figs/*.svg + flink-clients/src/main/resources/web-docs/js/*d3.js + + + **/packaged_licenses/LICENSE.*.txt + **/licenses/LICENSE* + **/licenses-binary/LICENSE* + + + flink-runtime-web/web-dashboard/package.json + flink-runtime-web/web-dashboard/package-lock.json + flink-runtime-web/web-dashboard/angular.json + flink-runtime-web/web-dashboard/proxy.conf.json + flink-runtime-web/web-dashboard/tsconfig.json + flink-runtime-web/web-dashboard/tslint.json + flink-runtime-web/web-dashboard/src/browserslist + flink-runtime-web/web-dashboard/src/tsconfig.app.json + flink-runtime-web/web-dashboard/src/tsconfig.spec.json + flink-runtime-web/web-dashboard/src/tslint.json + + + flink-runtime-web/web-dashboard/src/assets/** + + + flink-runtime-web/web-dashboard/web/** + + + flink-runtime-web/web-dashboard/node_modules/** + flink-runtime-web/web-dashboard/node/** + + + flink-table/flink-table-code-splitter/src/main/antlr4/** + + + **/src/test/resources/*-data + flink-tests/src/test/resources/testdata/terainput.txt + flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations + flink-scala/src/test/resources/flink_11-kryo_registrations + flink-core/src/test/resources/kryo-serializer-config-snapshot-v1 + flink-formats/flink-avro/src/test/resources/avro/*.avsc + out/test/flink-avro/avro/user.avsc + flink-table/flink-sql-client/src/test/resources/*.out + flink-table/flink-table-planner/src/test/resources/**/*.out + flink-table/flink-table-planner/src/test/resources/json/*.json + flink-yarn/src/test/resources/krb5.keytab + flink-end-to-end-tests/test-scripts/test-data/** + flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/keystore.jks + flink-connectors/flink-connector-kafka/src/test/resources/** + flink-connectors/flink-connector-hive/src/test/resources/** + flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/answer_set/* + flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/query/* + flink-connectors/flink-connector-aws-base/src/test/resources/profile + flink-connectors/flink-connector-kinesis/src/test/resources/profile + flink-connectors/flink-connector-aws-kinesis-streams/src/test/resources/profile + flink-table/flink-table-code-splitter/src/test/resources/** + flink-connectors/flink-connector-pulsar/src/test/resources/** + + + **/archunit-violations/** + + + **/src/test/resources/serializer-snapshot-* + **/src/test/resources/**/serializer-snapshot + **/src/test/resources/**/test-data + **/src/test/resources/*-snapshot + **/src/test/resources/*.snapshot + **/src/test/resources/*-savepoint/** + **/src/test/resources/*-savepoint-native/** + **/src/test/resources/*-checkpoint/** + flink-core/src/test/resources/serialized-kryo-serializer-1.3 + flink-core/src/test/resources/type-without-avro-serialized-using-kryo + flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized + + flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/avro/generated/* + flink-end-to-end-tests/flink-state-evolution-test/savepoints/* + flink-formats/flink-avro/src/test/resources/testdata.avro + flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/*.java + flink-formats/flink-avro-confluent-registry/src/test/resources/*.json + flink-formats/flink-avro-confluent-registry/src/test/resources/*.avro + flink-formats/flink-json/src/test/resources/*.txt + flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/*.java + flink-formats/flink-parquet/src/test/resources/avro/** + flink-formats/flink-parquet/src/test/resources/protobuf/** + + flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java + + **/flink-bin/conf/workers + **/flink-bin/conf/masters + + **/README.md + .github/** + + **/*.iml + flink-quickstart/**/testArtifact/goal.txt + + out/** + **/target/** + **/scalastyle-output.xml + build-target/** + docs/layouts/shortcodes/generated/** + docs/static/generated/** + + tools/artifacts/** + tools/flink*/** + + tools/japicmp-output/** + + tools/releasing/release/** + + apache-maven-3.2.5/** + + **/.idea/** + + flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/example/avro/** + flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/avro/** + + flink-jepsen/store/** + flink-jepsen/docker/id_rsa* + flink-jepsen/docker/nodes + + + flink-python/lib/** + flink-python/dev/download/** + flink-python/docs/_build/** + + + **/awssdk/global/handlers/execution.interceptors + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M5 + + + + ${flink.forkCount} + ${flink.reuseForks} + false + + 0${surefire.forkNumber} + true + true + + random + ${project.basedir} + + ${test.randomization.seed} + true + + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC -Duser.country=US -Duser.language=en + + + + + default-test + test + + test + + + + ${test.unit.pattern} + + + + + + integration-tests + integration-test + + test + + + + **/*.* + + + ${test.unit.pattern} + + **/*$* + + false + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + + org.eclipse.jdt.launching.JRE_CONTAINER + + + true + true + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + [3.1.1,) + + + ${target.java.version} + + + + + + ban-unsafe-snakeyaml + + enforce + + + + + + org.yaml:snakeyaml:(,1.26] + + + + org.yaml:snakeyaml:(,1.26]:*:test + + Older snakeyaml versions are not allowed due to security vulnerabilities. + + + + + + ban-unsafe-jackson + + enforce + + + + + + com.fasterxml.jackson*:*:(,2.12.0] + + Older jackson versions are not allowed due to security vulnerabilities. + + + + + + forbid-log4j-1 + + enforce + + + + + + log4j:log4j + org.slf4j:slf4j-log4j12 + + Log4j 1 dependencies are not allowed because they conflict with Log4j 2. If the dependency absolutely requires the Log4j 1 API, use 'org.apache.logging.log4j:log4j-1.2-api'. + + + + + + forbid-direct-akka-rpc-dependencies + + enforce + + + + + + org.apache.flink:flink-rpc-akka + + + Direct dependencies on flink-rpc-akka are not allowed. Depend on flink-rpc-akka-loader instead, and use RpcSystem#load or the TestingRpcService. + + + + + + + forbid-direct-table-planner-dependencies + + enforce + + + + + + org.apache.flink:flink-table-planner_${scala.binary.version} + + + org.apache.flink:flink-table-planner_${scala.binary.version}:*:*:test + + + Direct dependencies on flink-table-planner are not allowed. + You should depend on either Table API modules or flink-table-planner-loader. + + + + + + + dependency-convergence + + none + + enforce + + + + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + * + + + log4j.properties + log4j2.properties + log4j-test.properties + log4j2-test.properties + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + **/META-INF/maven/?*/?*/** + + + + + + + + + Apache Flink + UTF-8 + + + + + + shade-flink + package + + shade + + + false + false + true + ${project.basedir}/target/dependency-reduced-pom.xml + + + + + org.apache.flink:flink-shaded-force-shading + + ** + + + + + io.netty:netty + + META-INF/LICENSE.txt + + + + + + + org.apache.flink:flink-shaded-force-shading + + + + + + + + + + org.commonjava.maven.plugins + directory-maven-plugin + 0.1 + + + directories + + directory-of + + initialize + + rootDir + + org.apache.flink + flink-connectors + + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + ${target.java.version} + ${target.java.version} + + false + + + -Xpkginfo:always + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + + 8.14 + + + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + true + /tools/maven/checkstyle.xml + true + true + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.7 + + + + + + org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\# + + + + + + + + spotless-check + validate + + check + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + true + false + + -Xdoclint:none + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.0.0-M1 + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.2.0 + + + + org.apache.flink:* + + org.hamcrest:hamcrest-core + + org.powermock:powermock-core + org.powermock:powermock-reflect + org.powermock:powermock-api-support + + + + org.apache.flink:force-shading + + com.google.code.findbugs:jsr305 + org.scala-lang:scala-compiler + + org.slf4j:slf4j-api + + log4j:log4j + org.slf4j:slf4j-log4j12 + + org.apache.logging.log4j:log4j-slf4j-impl + org.apache.logging.log4j:log4j-api + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-1.2-api + + org.apache.flink:flink-test-utils-junit + junit:junit + org.mockito:mockito-core + org.powermock:powermock-api-mockito2 + org.powermock:powermock-module-junit4 + org.hamcrest:hamcrest-all + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + + + org.apache.maven.plugins + maven-site-plugin + + + attach-descriptor + none + + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + + validate + + check + + + + + false + true + true + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + ${project.basedir}/target/scalastyle-output.xml + UTF-8 + UTF-8 + + + + + + net.alchim31.maven + scala-maven-plugin + 3.2.2 + + + -nobootcp + -target:jvm-${target.java.version} + + + -Xss2m + + + + + + + com.github.siom79.japicmp + japicmp-maven-plugin + 0.11.0 + + + + org.apache.flink + ${project.artifactId} + ${japicmp.referenceVersion} + ${project.packaging} + + + + + ${project.build.directory}/${project.artifactId}-${project.version}.${project.packaging} + + + + true + + @org.apache.flink.annotation.Public + + + + + @org.apache.flink.annotation.Experimental + @org.apache.flink.annotation.PublicEvolving + @org.apache.flink.annotation.Internal + org.apache.flink.streaming.api.datastream.DataStream#DataStream(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment,org.apache.flink.streaming.api.transformations.StreamTransformation) + org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment + org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object) + org.apache.flink.streaming.api.functions.sink.SinkFunction + org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopOutputFormat + + public + false + true + true + false + true + false + true + + true + + ${rootDir}/${japicmp.outputDir}/${project.artifactId} + + + org.apache.flink + flink-annotations + ${project.version} + + + + + + verify + + cmp + + + + + + + + org.owasp + dependency-check-maven + 5.0.0-M2 + + ALL + true + true + + *flink-docs + *flink-end-to-end-tests + *flink-fs-tests* + *flink-yarn-tests* + + + + + + + diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml new file mode 100644 index 00000000..2048fd18 --- /dev/null +++ b/tools/maven/checkstyle.xml @@ -0,0 +1,562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml new file mode 100644 index 00000000..53b10529 --- /dev/null +++ b/tools/maven/scalastyle-config.xml @@ -0,0 +1,146 @@ + + + + + + + + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/spotbugs-exclude.xml b/tools/maven/spotbugs-exclude.xml new file mode 100644 index 00000000..b165464b --- /dev/null +++ b/tools/maven/spotbugs-exclude.xml @@ -0,0 +1,459 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + D + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml new file mode 100644 index 00000000..129a0c7a --- /dev/null +++ b/tools/maven/suppressions.xml @@ -0,0 +1,85 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +