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.
+ *
+ *
+ *
+ * @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 super IN> emitter;
+ private final BulkProcessorConfig buildBulkProcessorConfig;
+ private final BulkProcessorBuilderFactory bulkProcessorBuilderFactory;
+ private final NetworkClientConfig networkClientConfig;
+ private final DeliveryGuarantee deliveryGuarantee;
+
+ ElasticsearchSink(
+ List hosts,
+ ElasticsearchEmitter super IN> 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 super IN> 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 super T> 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 super IN> 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 super IN> 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 extends Exception> 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.
+ *
+ *
+ *
+ * @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 extends Exception> 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