├── .editorconfig ├── .github ├── ISSUE_TEMPLATE │ ├── bug_report.yml │ └── sub_task.yml ├── PULL_REQUEST_TEMPLATE └── workflows │ └── ci.yml ├── .gitignore ├── .mvn └── wrapper │ └── maven-wrapper.properties ├── LICENSE ├── README.md ├── docs ├── content.zh │ └── docs │ │ └── connectors │ │ └── .keep └── content │ └── docs │ └── connectors │ └── .keep ├── flink-connector-clickhouse-e2e-test ├── pom.xml └── src │ └── test │ └── java │ └── org │ └── apache │ └── flink │ └── connector │ └── clickhouse │ ├── ClickhouseE2ECase.java │ ├── ClickhouseProxy.java │ └── FlinkContainerTestEnvironment.java ├── flink-connector-clickhouse ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── clickhouse │ │ │ ├── ClickHouseDynamicTableFactory.java │ │ │ ├── ClickHouseDynamicTableSink.java │ │ │ ├── ClickHouseDynamicTableSource.java │ │ │ ├── catalog │ │ │ ├── ClickHouseCatalog.java │ │ │ └── ClickHouseCatalogFactory.java │ │ │ ├── config │ │ │ ├── ClickHouseConfig.java │ │ │ └── ClickHouseConfigOptions.java │ │ │ ├── internal │ │ │ ├── AbstractClickHouseInputFormat.java │ │ │ ├── AbstractClickHouseOutputFormat.java │ │ │ ├── ClickHouseBatchInputFormat.java │ │ │ ├── ClickHouseBatchOutputFormat.java │ │ │ ├── ClickHouseRowDataLookupFunction.java │ │ │ ├── ClickHouseRowDataSinkFunction.java │ │ │ ├── ClickHouseShardInputFormat.java │ │ │ ├── ClickHouseShardOutputFormat.java │ │ │ ├── ClickHouseStatementFactory.java │ │ │ ├── connection │ │ │ │ ├── ClickHouseConnectionProvider.java │ │ │ │ ├── ClickHouseStatementWrapper.java │ │ │ │ └── ObjectArray.java │ │ │ ├── converter │ │ │ │ ├── ClickHouseConverterUtils.java │ │ │ │ └── ClickHouseRowConverter.java │ │ │ ├── executor │ │ │ │ ├── ClickHouseBatchExecutor.java │ │ │ │ ├── ClickHouseExecutor.java │ │ │ │ └── ClickHouseUpsertExecutor.java │ │ │ ├── options │ │ │ │ ├── ClickHouseConnectionOptions.java │ │ │ │ ├── ClickHouseDmlOptions.java │ │ │ │ └── ClickHouseReadOptions.java │ │ │ ├── partitioner │ │ │ │ ├── BalancedPartitioner.java │ │ │ │ ├── ClickHousePartitioner.java │ │ │ │ ├── JavaHashPartitioner.java │ │ │ │ ├── ShufflePartitioner.java │ │ │ │ └── ValuePartitioner.java │ │ │ └── schema │ │ │ │ ├── ClusterSpec.java │ │ │ │ ├── DistributedEngineFull.java │ │ │ │ ├── Expression.java │ │ │ │ ├── FieldExpr.java │ │ │ │ ├── FunctionExpr.java │ │ │ │ ├── ReplicaSpec.java │ │ │ │ └── ShardSpec.java │ │ │ ├── split │ │ │ ├── ClickHouseBatchBetweenParametersProvider.java │ │ │ ├── ClickHouseBetweenParametersProvider.java │ │ │ ├── ClickHouseNonParametersProvider.java │ │ │ ├── ClickHouseParametersProvider.java │ │ │ ├── ClickHouseShardBetweenParametersProvider.java │ │ │ └── ClickHouseShardTableParametersProvider.java │ │ │ └── util │ │ │ ├── ClickHouseJdbcUtil.java │ │ │ ├── ClickHouseUtil.java │ │ │ ├── DataTypeUtil.java │ │ │ ├── FilterPushDownHelper.java │ │ │ └── SqlClause.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.table.factories.Factory │ └── test │ └── java │ └── org │ └── apache │ └── flink │ └── connector │ └── clickhouse │ └── AppTest.java ├── flink-sql-connector-clickhouse └── pom.xml ├── mvnw ├── mvnw.cmd ├── pom.xml └── tools ├── maven ├── checkstyle.xml └── suppressions.xml └── releasing └── shared ├── .gitignore ├── LICENSE ├── README.md ├── _init.sh ├── _utils.sh ├── check_environment.sh ├── release_git_tag.sh ├── release_snapshot_branch.sh ├── release_source_release.sh ├── stage_jars.sh ├── stage_source_release.sh ├── update_branch_version.sh └── update_japicmp_configuration.sh /.github/ISSUE_TEMPLATE/bug_report.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | name: Apache Flink Clickhouse Connector Bug Report 18 | description: File a bug report 19 | title: "[Bug]: " 20 | labels: ["type:bug"] 21 | body: 22 | - type: markdown 23 | attributes: 24 | value: | 25 | Thanks for taking the time to fill out this bug report! 26 | We really appreciate the community's efforts to improve Clickhouse Connector. 27 | 28 | Please feel free to report the problem you encountered. 29 | If you're sure that it is indeed a bug, please try your best to log the reproducible steps. 30 | 31 | - type: textarea 32 | id: what-happened 33 | attributes: 34 | label: What happened? 35 | description: Also tell us, what did you expect to happen? 36 | placeholder: Please provide the context in which the problem occurred and explain what happened 37 | validations: 38 | required: true 39 | 40 | - type: input 41 | id: version 42 | attributes: 43 | label: Affects Versions 44 | description: What version of Clickhouse Connector are affected by this bug? 45 | placeholder: > 46 | e.g. master/1.18/1.19/1.20/... 47 | validations: 48 | required: true 49 | 50 | - type: dropdown 51 | id: engines 52 | attributes: 53 | label: What are you seeing the problem on? 54 | multiple: true 55 | options: 56 | - Clickhouse-Client 57 | - Connector Connection 58 | - Flink-Table-Api (SQL) 59 | - Flink-DataStream-Api 60 | - Ci 61 | - Docs 62 | 63 | - type: textarea 64 | id: reproduce 65 | attributes: 66 | label: How to reproduce 67 | description: > 68 | What should we do to reproduce the problem? 69 | placeholder: > 70 | Please make sure you provide a reproducible step-by-step case of how to reproduce the problem 71 | as minimally and precisely as possible. Keep in mind we do not have access to your cluster. 72 | Remember that non-reproducible issues will be closed! Opening a discussion is recommended as a 73 | first step. 74 | 75 | - type: textarea 76 | id: logs 77 | attributes: 78 | label: Relevant log output 79 | description: Please copy and paste any relevant log output. This will be automatically formatted into code, so no need for backticks. 80 | render: shell 81 | 82 | - type: textarea 83 | attributes: 84 | label: Anything else 85 | description: Anything else we need to know? 86 | placeholder: > 87 | e.g How often does this problem occur? (Once? Every time? Only when certain conditions are met?) 88 | validations: 89 | required: false 90 | 91 | - type: checkboxes 92 | attributes: 93 | label: Are you willing to submit a PR? 94 | description: > 95 | This is absolutely not required, but we are happy to guide you in the contribution process 96 | especially if you already have a good understanding of how to fix the bug. 97 | Connector is a community-managed project and we love to bring new contributors in. 98 | options: 99 | - label: Yes I am willing to submit a PR! 100 | 101 | - type: checkboxes 102 | id: terms 103 | attributes: 104 | label: Code of Conduct 105 | description: By submitting this issue, you agree to follow our [Code of Conduct](https://www.apache.org/foundation/policies/conduct) 106 | options: 107 | - label: I agree to follow this project's Code of Conduct 108 | required: true 109 | 110 | - type: markdown 111 | attributes: 112 | value: "Thanks for completing our form!" 113 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/sub_task.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | name: Apache Flink Clickhouse Connector Subtask 18 | title: "[Subtask]: " 19 | description: Subtask of another issue 20 | labels: [ "type:subtask" ] 21 | body: 22 | - type: markdown 23 | attributes: 24 | value: | 25 | Thank you for contributing to Connector! 26 | 27 | - type: checkboxes 28 | attributes: 29 | label: Search before asking 30 | options: 31 | - label: > 32 | I have searched in the [issues](https://github.com/itinycheng/flink-connector-clickhouse/issues?q=is%3Aissue) and found no similar 33 | issues. 34 | required: true 35 | 36 | - type: textarea 37 | attributes: 38 | label: Description 39 | description: A short description of your subtask 40 | 41 | - type: textarea 42 | attributes: 43 | label: Parent issue 44 | placeholder: > 45 | Please put the parent issue link here. 46 | validations: 47 | required: true 48 | 49 | 50 | - type: checkboxes 51 | attributes: 52 | label: Are you willing to submit PR? 53 | description: > 54 | This is absolutely not required, but we are happy to guide you in the contribution process 55 | especially if you already have a good understanding of how to implement the feature. 56 | Connector is a community-managed project and we love to bring new contributors in. 57 | options: 58 | - label: Yes I am willing to submit a PR! 59 | 60 | - type: checkboxes 61 | attributes: 62 | label: Code of Conduct 63 | description: The Code of Conduct helps create a safe space for everyone. We require 64 | that everyone agrees to it. 65 | options: 66 | - label: > 67 | I agree to follow this project's 68 | [Code of Conduct](https://www.apache.org/foundation/policies/conduct) 69 | required: true 70 | 71 | - type: markdown 72 | attributes: 73 | value: "Thanks for completing our form!" 74 | -------------------------------------------------------------------------------- /.github/PULL_REQUEST_TEMPLATE: -------------------------------------------------------------------------------- 1 | 10 | 11 | ## Why are the changes needed? 12 | 13 | 19 | 20 | Close #xxx. 21 | 22 | ## Brief change log 23 | 26 | 27 | - 28 | 29 | ## How was this patch tested? 30 | 31 | - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible 32 | 33 | - [ ] Add screenshots for manual tests if appropriate 34 | 35 | - [ ] Run test locally before making a pull request 36 | 37 | ## Documentation 38 | 39 | - Does this pull request introduce a new feature? (yes / no) 40 | - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented) 41 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | name: CI 17 | on: 18 | push: 19 | branches: [ master ] 20 | pull_request: 21 | branches: [ master ] 22 | jobs: 23 | maven_verify: 24 | strategy: 25 | matrix: 26 | java-version: [ 8, 11 ] 27 | runs-on: [ ubuntu-latest ] 28 | runs-on: ${{ matrix.runs-on }} 29 | steps: 30 | - name: Checkout 31 | uses: actions/checkout@v4 32 | - name: Set up JDK ${{ matrix.java-version }} 33 | uses: actions/setup-java@v4 34 | with: 35 | java-version: ${{ matrix.java-version }} 36 | distribution: temurin 37 | cache: maven 38 | - name: Set up Docker Buildx 39 | uses: docker/setup-buildx-action@v3.10.0 40 | - name: Build with Maven 41 | run: ./mvnw verify 42 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Created by .ignore support plugin (hsz.mobi) 2 | ### Java template 3 | # Compiled class file 4 | *.class 5 | 6 | # Log file 7 | *.log 8 | 9 | # BlueJ files 10 | *.ctxt 11 | 12 | # Mobile Tools for Java (J2ME) 13 | .mtj.tmp/ 14 | 15 | # Package Files # 16 | *.jar 17 | *.war 18 | *.nar 19 | *.ear 20 | *.zip 21 | *.tar.gz 22 | *.rar 23 | 24 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 25 | hs_err_pid* 26 | 27 | ### Maven template 28 | target/ 29 | pom.xml.tag 30 | pom.xml.releaseBackup 31 | pom.xml.versionsBackup 32 | pom.xml.next 33 | release.properties 34 | dependency-reduced-pom.xml 35 | buildNumber.properties 36 | .mvn/timing.properties 37 | # https://github.com/takari/maven-wrapper#usage-without-binary-jar 38 | .mvn/wrapper/maven-wrapper.jar 39 | .idea/ 40 | *.iml 41 | 42 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.4/apache-maven-3.8.4-bin.zip 18 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar 19 | -------------------------------------------------------------------------------- /docs/content.zh/docs/connectors/.keep: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itinycheng/flink-connector-clickhouse/5642f6930f0f27c974b9f73ce1b2497c235f5872/docs/content.zh/docs/connectors/.keep -------------------------------------------------------------------------------- /docs/content/docs/connectors/.keep: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/itinycheng/flink-connector-clickhouse/5642f6930f0f27c974b9f73ce1b2497c235f5872/docs/content/docs/connectors/.keep -------------------------------------------------------------------------------- /flink-connector-clickhouse-e2e-test/pom.xml: -------------------------------------------------------------------------------- 1 | 17 | 20 | 4.0.0 21 | 22 | org.apache.flink 23 | flink-connector-clickhouse-parent 24 | 1.0.0-SNAPSHOT 25 | 26 | 27 | Flink : Connectors : Clickhouse : E2E Tests 28 | flink-connector-clickhouse-e2e-test 29 | https://flink.apache.org 30 | jar 31 | 32 | 33 | UTF-8 34 | 35 | 36 | 37 | 38 | org.testcontainers 39 | clickhouse 40 | ${testcontainer.version} 41 | 42 | 43 | org.apache.flink 44 | flink-connector-test-utils 45 | ${flink.version} 46 | test 47 | 48 | 49 | org.apache.flink 50 | flink-connector-clickhouse 51 | ${project.version} 52 | test 53 | 54 | 55 | 56 | 57 | 58 | org.apache.maven.plugins 59 | maven-surefire-plugin 60 | 61 | 62 | default-test 63 | none 64 | 65 | 66 | integration-tests 67 | none 68 | 69 | 70 | end-to-end-tests 71 | integration-test 72 | 73 | test 74 | 75 | 76 | 77 | **/*.* 78 | 79 | 1 80 | 81 | ${project.basedir} 82 | 83 | 84 | 85 | 86 | 87 | 88 | org.apache.maven.plugins 89 | maven-dependency-plugin 90 | 91 | 92 | copy-jars 93 | process-resources 94 | 95 | copy 96 | 97 | 98 | 99 | 100 | 101 | 102 | org.apache.flink 103 | flink-connector-clickhouse 104 | ${project.version} 105 | jar 106 | true 107 | ${project.build.directory} 108 | 109 | 110 | org.apache.httpcomponents.client5 111 | httpclient5 112 | 5.2.1 113 | jar 114 | true 115 | ${project.build.directory} 116 | 117 | 118 | org.apache.httpcomponents.core5 119 | httpcore5 120 | 5.2 121 | jar 122 | true 123 | ${project.build.directory} 124 | 125 | 126 | org.apache.httpcomponents.core5 127 | httpcore5-h2 128 | 5.2 129 | jar 130 | true 131 | ${project.build.directory} 132 | 133 | 134 | com.clickhouse 135 | clickhouse-jdbc 136 | ${clickhouse-jdbc.version} 137 | jar 138 | true 139 | ${project.build.directory} 140 | 141 | 142 | 143 | 144 | 145 | 146 | 147 | -------------------------------------------------------------------------------- /flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseE2ECase.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.clickhouse; 2 | 3 | import org.junit.After; 4 | import org.junit.Test; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | import java.time.Duration; 9 | import java.time.temporal.ChronoUnit; 10 | import java.util.ArrayList; 11 | import java.util.Arrays; 12 | import java.util.List; 13 | 14 | /** End-to-end test for Clickhouse. */ 15 | public class ClickhouseE2ECase extends FlinkContainerTestEnvironment { 16 | 17 | private static final Logger logger = LoggerFactory.getLogger(ClickhouseE2ECase.class); 18 | 19 | ClickhouseProxy proxy; 20 | 21 | @Test 22 | public void testSink() throws Exception { 23 | String jdbcUrl = String.format("jdbc:clickhouse://%s:%s/default", "clickhouse", "8123"); 24 | 25 | proxy = 26 | new ClickhouseProxy( 27 | CLICKHOUSE_CONTAINER.getJdbcUrl(), 28 | CLICKHOUSE_CONTAINER.getUsername(), 29 | CLICKHOUSE_CONTAINER.getPassword()); 30 | proxy.execute( 31 | "create table test (id Int32, name String, float32_column Float32, date_column Date,datetime_column DateTime, array_column Array(Int32)) engine = Memory"); 32 | proxy.execute( 33 | "create table test_insert (id Int32, name String, float32_column Float32, date_column Date,datetime_column DateTime, array_column Array(Int32)) engine = Memory; "); 34 | proxy.execute( 35 | "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (1, 'Name1', 1.1, '2022-01-01', '2022-01-01 00:00:00', [1, 2, 3]);"); 36 | proxy.execute( 37 | "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (2, 'Name2', 2.2, '2022-01-02', '2022-01-02 01:00:00', [4, 5, 6]);"); 38 | proxy.execute( 39 | "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (3, 'Name3', 3.3, '2022-01-03', '2022-01-03 02:00:00', [7, 8, 9]);"); 40 | proxy.execute( 41 | "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (4, 'Name4', 4.4, '2022-01-04', '2022-01-04 03:00:00', [10, 11, 12]);"); 42 | proxy.execute( 43 | "INSERT INTO test (id, name, float32_column, date_column, datetime_column, array_column) VALUES (5, 'Name5', 5.5, '2022-01-05', '2022-01-05 04:00:00', [13, 14, 15]);"); 44 | // proxy.execute("insert into test values (2, 'kiki');"); 45 | List sqlLines = new ArrayList<>(); 46 | sqlLines.add( 47 | "create table clickhouse_test (id int, name varchar,float32_column FLOAT,\n" 48 | + " datetime_column TIMESTAMP(3),\n" 49 | + " array_column ARRAY) with ('connector' = 'clickhouse',\n" 50 | + " 'url' = '" 51 | + jdbcUrl 52 | + "',\n" 53 | + " 'table-name' = 'test',\n" 54 | + " 'username'='test_username',\n" 55 | + " 'password'='test_password'\n" 56 | + ");"); 57 | sqlLines.add( 58 | "create table test (id int, name varchar,float32_column FLOAT,\n" 59 | + " datetime_column TIMESTAMP(3),\n" 60 | + " array_column ARRAY) with ('connector' = 'clickhouse',\n" 61 | + " 'url' = '" 62 | + jdbcUrl 63 | + "',\n" 64 | + " 'table-name' = 'test_insert',\n" 65 | + " 'username'='test_username',\n" 66 | + " 'password'='test_password'\n" 67 | + ");"); 68 | sqlLines.add("insert into test select * from clickhouse_test;"); 69 | 70 | submitSQLJob( 71 | sqlLines, 72 | SQL_CONNECTOR_CLICKHOUSE_JAR, 73 | CLICKHOUSE_JDBC_JAR, 74 | HTTPCORE_JAR, 75 | HTTPCLIENT_JAR, 76 | HTTPCLIENT_H2_JAR); 77 | waitUntilJobRunning(Duration.of(1, ChronoUnit.MINUTES)); 78 | List expectedResult = 79 | Arrays.asList( 80 | "1,Name1,1.1,2022-01-01 00:00:00,[1,2,3]", 81 | "2,Name2,2.2,2022-01-02 01:00:00,[4,5,6]", 82 | "3,Name3,3.3,2022-01-03 02:00:00,[7,8,9]", 83 | "4,Name4,4.4,2022-01-04 03:00:00,[10,11,12]", 84 | "5,Name5,5.5,2022-01-05 04:00:00,[13,14,15]"); 85 | proxy.checkResultWithTimeout( 86 | expectedResult, 87 | "test_insert", 88 | Arrays.asList("id", "name", "float32_column", "datetime_column", "array_column"), 89 | 60000); 90 | } 91 | 92 | @After 93 | public void tearDown() { 94 | CLICKHOUSE_CONTAINER.stop(); 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /flink-connector-clickhouse-e2e-test/src/test/java/org/apache/flink/connector/clickhouse/ClickhouseProxy.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.clickhouse; 2 | 3 | import com.clickhouse.jdbc.ClickHouseConnection; 4 | import com.clickhouse.jdbc.ClickHouseDataSource; 5 | import com.clickhouse.jdbc.ClickHouseDriver; 6 | import com.clickhouse.jdbc.ClickHouseStatement; 7 | import org.junit.Assert; 8 | import org.slf4j.Logger; 9 | import org.slf4j.LoggerFactory; 10 | 11 | import java.sql.Array; 12 | import java.sql.ResultSet; 13 | import java.sql.ResultSetMetaData; 14 | import java.sql.SQLException; 15 | import java.sql.Timestamp; 16 | import java.util.ArrayList; 17 | import java.util.Collections; 18 | import java.util.List; 19 | import java.util.Properties; 20 | 21 | /** A proxy for Clickhouse to execute SQLs and check results. */ 22 | public class ClickhouseProxy { 23 | private final String jdbcUrl; 24 | private final String username; 25 | private final String password; 26 | private static final Logger logger = LoggerFactory.getLogger(ClickhouseProxy.class); 27 | ClickHouseDriver driver; 28 | ClickHouseStatement statement; 29 | ClickHouseConnection connection; 30 | 31 | ClickhouseProxy(String jdbcUrl, String username, String password) { 32 | this.jdbcUrl = jdbcUrl; 33 | this.username = username; 34 | this.password = password; 35 | this.driver = new ClickHouseDriver(); 36 | } 37 | 38 | public void connect() { 39 | try { 40 | if (connection == null) { 41 | Properties properties = new Properties(); 42 | properties.put("username", username); 43 | properties.put("password", password); 44 | ClickHouseDataSource clickHouseDataSource = 45 | new ClickHouseDataSource(jdbcUrl, properties); 46 | connection = clickHouseDataSource.getConnection(username, password); 47 | statement = connection.createStatement(); 48 | } 49 | } catch (Exception e) { 50 | logger.error("Failed to connect to clickhouse", e); 51 | } 52 | } 53 | 54 | public void execute(String sql) throws SQLException { 55 | connect(); 56 | statement.execute(sql); 57 | } 58 | 59 | private void checkResult(List expectedResult, String table, List fields) 60 | throws Exception { 61 | connect(); 62 | List results = new ArrayList<>(); 63 | ResultSet resultSet = statement.executeQuery("select * from " + table); 64 | while (resultSet.next()) { 65 | List result = new ArrayList<>(); 66 | ResultSetMetaData metaData = resultSet.getMetaData(); 67 | int columnCount = metaData.getColumnCount(); 68 | for (int i = 1; i <= columnCount; i++) { 69 | String columnName = metaData.getColumnName(i); 70 | if (!fields.contains(columnName)) { 71 | continue; 72 | } 73 | String columnType = metaData.getColumnTypeName(i); 74 | switch (columnType) { 75 | case "Array": 76 | Array array = resultSet.getArray(i); 77 | result.add(array.toString()); 78 | break; 79 | case "Timestamp": 80 | Timestamp timestamp = resultSet.getTimestamp(i); 81 | result.add(timestamp.toString()); 82 | break; 83 | default: 84 | String value = resultSet.getString(i); 85 | result.add(value); 86 | break; 87 | } 88 | } 89 | 90 | results.add(String.join(",", result)); 91 | } 92 | Collections.sort(results); 93 | Collections.sort(expectedResult); 94 | Assert.assertArrayEquals(expectedResult.toArray(), results.toArray()); 95 | } 96 | 97 | public void checkResultWithTimeout( 98 | List expectedResult, String table, List fields, long timeout) 99 | throws Exception { 100 | long endTimeout = System.currentTimeMillis() + timeout; 101 | boolean result = false; 102 | while (System.currentTimeMillis() < endTimeout) { 103 | try { 104 | checkResult(expectedResult, table, fields); 105 | result = true; 106 | break; 107 | } catch (AssertionError | SQLException throwable) { 108 | Thread.sleep(1000L); 109 | } 110 | } 111 | if (!result) { 112 | checkResult(expectedResult, table, fields); 113 | } 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/pom.xml: -------------------------------------------------------------------------------- 1 | 17 | 20 | 4.0.0 21 | 22 | org.apache.flink 23 | flink-connector-clickhouse-parent 24 | 1.0.0-SNAPSHOT 25 | 26 | 27 | flink-connector-clickhouse 28 | Flink : Connectors : Clickhouse 29 | jar 30 | 31 | 32 | 33 | org.apache.flink 34 | flink-connector-base 35 | 36 | 37 | org.apache.flink 38 | flink-table-api-java-bridge 39 | 40 | 41 | 42 | 43 | com.clickhouse 44 | clickhouse-jdbc 45 | 46 | 47 | org.apache.commons 48 | commons-lang3 49 | 50 | 51 | org.apache.httpcomponents.client5 52 | httpclient5 53 | 54 | 55 | 56 | 57 | org.apache.flink 58 | flink-test-utils 59 | ${flink.version} 60 | test 61 | 62 | 63 | org.apache.flink 64 | flink-connector-test-utils 65 | ${flink.version} 66 | test 67 | 68 | 69 | org.apache.flink 70 | flink-streaming-java 71 | ${flink.version} 72 | test 73 | test-jar 74 | 75 | 76 | org.apache.flink 77 | flink-runtime 78 | ${flink.version} 79 | test-jar 80 | test 81 | 82 | 83 | org.apache.flink 84 | flink-connector-base 85 | ${flink.version} 86 | test 87 | test-jar 88 | 89 | 90 | 91 | 92 | org.apache.flink 93 | flink-table-common 94 | ${flink.version} 95 | test-jar 96 | test 97 | 98 | 99 | 100 | org.apache.flink 101 | flink-table-test-utils 102 | ${flink.version} 103 | test 104 | 105 | 106 | 107 | org.apache.flink 108 | flink-table-api-scala-bridge_${scala.binary.version} 109 | ${flink.version} 110 | test 111 | 112 | 113 | org.apache.flink 114 | flink-table-planner_${scala.binary.version} 115 | ${flink.version} 116 | test-jar 117 | test 118 | 119 | 120 | 121 | 122 | org.apache.flink 123 | flink-json 124 | ${flink.version} 125 | test 126 | 127 | 128 | org.apache.flink 129 | flink-avro 130 | ${flink.version} 131 | test 132 | 133 | 134 | org.apache.flink 135 | flink-avro-confluent-registry 136 | ${flink.version} 137 | test 138 | 139 | 140 | org.apache.flink 141 | flink-csv 142 | ${flink.version} 143 | test 144 | 145 | 146 | 147 | 148 | 149 | 150 | org.apache.maven.plugins 151 | maven-jar-plugin 152 | 153 | 154 | 155 | test-jar 156 | 157 | 158 | 159 | META-INF/LICENSE 160 | META-INF/NOTICE 161 | 162 | 163 | 164 | 165 | 166 | 167 | org.apache.maven.plugins 168 | maven-source-plugin 169 | 170 | 171 | attach-test-sources 172 | 173 | test-jar-no-fork 174 | 175 | 176 | 177 | 178 | false 179 | 180 | 181 | META-INF/LICENSE 182 | META-INF/NOTICE 183 | 184 | 185 | 186 | 187 | 188 | 189 | 190 | 191 | 192 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/ClickHouseDynamicTableSink.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.AbstractClickHouseOutputFormat; 21 | import org.apache.flink.connector.clickhouse.internal.ClickHouseRowDataSinkFunction; 22 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseDmlOptions; 23 | import org.apache.flink.table.connector.ChangelogMode; 24 | import org.apache.flink.table.connector.sink.DynamicTableSink; 25 | import org.apache.flink.table.connector.sink.SinkFunctionProvider; 26 | import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; 27 | import org.apache.flink.table.types.DataType; 28 | import org.apache.flink.types.RowKind; 29 | import org.apache.flink.util.Preconditions; 30 | 31 | import javax.annotation.Nonnull; 32 | 33 | import java.util.LinkedHashMap; 34 | import java.util.Map; 35 | import java.util.Properties; 36 | 37 | /** 38 | * A {@link DynamicTableSink} that describes how to create a {@link ClickHouseDynamicTableSink} from 39 | * a logical description. 40 | * 41 | *

TODO: Partitioning strategy isn't well implemented. 42 | */ 43 | public class ClickHouseDynamicTableSink implements DynamicTableSink, SupportsPartitioning { 44 | 45 | private final String[] primaryKeys; 46 | 47 | private final String[] partitionKeys; 48 | 49 | private final DataType physicalRowDataType; 50 | 51 | private final ClickHouseDmlOptions options; 52 | 53 | private final Properties connectionProperties; 54 | 55 | private boolean dynamicGrouping = false; 56 | 57 | private LinkedHashMap staticPartitionSpec = new LinkedHashMap<>(); 58 | 59 | public ClickHouseDynamicTableSink( 60 | @Nonnull ClickHouseDmlOptions options, 61 | @Nonnull Properties connectionProperties, 62 | @Nonnull String[] primaryKeys, 63 | @Nonnull String[] partitionKeys, 64 | @Nonnull DataType physicalRowDataType) { 65 | this.options = options; 66 | this.connectionProperties = connectionProperties; 67 | this.primaryKeys = primaryKeys; 68 | this.partitionKeys = partitionKeys; 69 | this.physicalRowDataType = physicalRowDataType; 70 | } 71 | 72 | @Override 73 | public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { 74 | validatePrimaryKey(requestedMode); 75 | return ChangelogMode.newBuilder() 76 | .addContainedKind(RowKind.INSERT) 77 | .addContainedKind(RowKind.UPDATE_AFTER) 78 | .addContainedKind(RowKind.DELETE) 79 | .build(); 80 | } 81 | 82 | private void validatePrimaryKey(ChangelogMode requestedMode) { 83 | Preconditions.checkState( 84 | ChangelogMode.insertOnly().equals(requestedMode) || primaryKeys.length > 0, 85 | "Please declare primary key for sink table when query contains update/delete record."); 86 | } 87 | 88 | @Override 89 | public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { 90 | AbstractClickHouseOutputFormat outputFormat = 91 | new AbstractClickHouseOutputFormat.Builder() 92 | .withOptions(options) 93 | .withConnectionProperties(connectionProperties) 94 | .withFieldNames( 95 | DataType.getFieldNames(physicalRowDataType).toArray(new String[0])) 96 | .withFieldTypes( 97 | DataType.getFieldDataTypes(physicalRowDataType) 98 | .toArray(new DataType[0])) 99 | .withPrimaryKey(primaryKeys) 100 | .withPartitionKey(partitionKeys) 101 | .build(); 102 | return SinkFunctionProvider.of( 103 | new ClickHouseRowDataSinkFunction(outputFormat), options.getParallelism()); 104 | } 105 | 106 | @Override 107 | public void applyStaticPartition(Map partition) { 108 | staticPartitionSpec = new LinkedHashMap<>(); 109 | for (String partitionCol : partitionKeys) { 110 | if (partition.containsKey(partitionCol)) { 111 | staticPartitionSpec.put(partitionCol, partition.get(partitionCol)); 112 | } 113 | } 114 | } 115 | 116 | @Override 117 | public boolean requiresPartitionGrouping(boolean supportsGrouping) { 118 | this.dynamicGrouping = supportsGrouping; 119 | return supportsGrouping; 120 | } 121 | 122 | @Override 123 | public DynamicTableSink copy() { 124 | ClickHouseDynamicTableSink sink = 125 | new ClickHouseDynamicTableSink( 126 | options, 127 | connectionProperties, 128 | primaryKeys, 129 | partitionKeys, 130 | physicalRowDataType); 131 | sink.dynamicGrouping = dynamicGrouping; 132 | sink.staticPartitionSpec = staticPartitionSpec; 133 | return sink; 134 | } 135 | 136 | @Override 137 | public String asSummaryString() { 138 | return "ClickHouse table sink"; 139 | } 140 | } 141 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/ClickHouseDynamicTableSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.AbstractClickHouseInputFormat; 21 | import org.apache.flink.connector.clickhouse.internal.ClickHouseRowDataLookupFunction; 22 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseReadOptions; 23 | import org.apache.flink.connector.clickhouse.util.FilterPushDownHelper; 24 | import org.apache.flink.table.connector.ChangelogMode; 25 | import org.apache.flink.table.connector.Projection; 26 | import org.apache.flink.table.connector.source.DynamicTableSource; 27 | import org.apache.flink.table.connector.source.InputFormatProvider; 28 | import org.apache.flink.table.connector.source.LookupTableSource; 29 | import org.apache.flink.table.connector.source.ScanTableSource; 30 | import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; 31 | import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; 32 | import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; 33 | import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider; 34 | import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider; 35 | import org.apache.flink.table.connector.source.lookup.cache.LookupCache; 36 | import org.apache.flink.table.expressions.ResolvedExpression; 37 | import org.apache.flink.table.types.DataType; 38 | import org.apache.flink.table.types.logical.RowType; 39 | import org.apache.flink.util.Preconditions; 40 | 41 | import javax.annotation.Nullable; 42 | 43 | import java.util.ArrayList; 44 | import java.util.List; 45 | import java.util.Properties; 46 | 47 | /** ClickHouse table source. */ 48 | public class ClickHouseDynamicTableSource 49 | implements ScanTableSource, 50 | LookupTableSource, 51 | SupportsProjectionPushDown, 52 | SupportsLimitPushDown, 53 | SupportsFilterPushDown { 54 | 55 | private final ClickHouseReadOptions readOptions; 56 | 57 | private final Properties connectionProperties; 58 | 59 | private final int lookupMaxRetryTimes; 60 | 61 | @Nullable private final LookupCache cache; 62 | 63 | private DataType physicalRowDataType; 64 | 65 | private String filterClause; 66 | 67 | private long limit = -1L; 68 | 69 | public ClickHouseDynamicTableSource( 70 | ClickHouseReadOptions readOptions, 71 | int lookupMaxRetryTimes, 72 | @Nullable LookupCache cache, 73 | Properties properties, 74 | DataType physicalRowDataType) { 75 | this.readOptions = readOptions; 76 | this.connectionProperties = properties; 77 | this.lookupMaxRetryTimes = lookupMaxRetryTimes; 78 | this.cache = cache; 79 | this.physicalRowDataType = physicalRowDataType; 80 | } 81 | 82 | @Override 83 | public ChangelogMode getChangelogMode() { 84 | return ChangelogMode.insertOnly(); 85 | } 86 | 87 | @Override 88 | public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { 89 | // ClickHouse only support non-nested look up keys 90 | String[] keyNames = new String[context.getKeys().length]; 91 | for (int i = 0; i < keyNames.length; i++) { 92 | int[] innerKeyArr = context.getKeys()[i]; 93 | Preconditions.checkArgument( 94 | innerKeyArr.length == 1, "ClickHouse only support non-nested look up keys"); 95 | keyNames[i] = DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]); 96 | } 97 | final RowType rowType = (RowType) physicalRowDataType.getLogicalType(); 98 | ClickHouseRowDataLookupFunction lookupFunction = 99 | new ClickHouseRowDataLookupFunction( 100 | readOptions, 101 | lookupMaxRetryTimes, 102 | DataType.getFieldNames(physicalRowDataType).toArray(new String[0]), 103 | DataType.getFieldDataTypes(physicalRowDataType).toArray(new DataType[0]), 104 | keyNames, 105 | rowType); 106 | if (cache != null) { 107 | return PartialCachingLookupProvider.of(lookupFunction, cache); 108 | } else { 109 | return LookupFunctionProvider.of(lookupFunction); 110 | } 111 | } 112 | 113 | @Override 114 | public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { 115 | AbstractClickHouseInputFormat.Builder builder = 116 | new AbstractClickHouseInputFormat.Builder() 117 | .withOptions(readOptions) 118 | .withConnectionProperties(connectionProperties) 119 | .withFieldNames( 120 | DataType.getFieldNames(physicalRowDataType).toArray(new String[0])) 121 | .withFieldTypes( 122 | DataType.getFieldDataTypes(physicalRowDataType) 123 | .toArray(new DataType[0])) 124 | .withRowDataTypeInfo( 125 | runtimeProviderContext.createTypeInformation(physicalRowDataType)) 126 | .withFilterClause(filterClause) 127 | .withLimit(limit); 128 | return InputFormatProvider.of(builder.build()); 129 | } 130 | 131 | @Override 132 | public DynamicTableSource copy() { 133 | ClickHouseDynamicTableSource source = 134 | new ClickHouseDynamicTableSource( 135 | readOptions, 136 | lookupMaxRetryTimes, 137 | cache, 138 | connectionProperties, 139 | physicalRowDataType); 140 | source.filterClause = filterClause; 141 | source.limit = limit; 142 | return source; 143 | } 144 | 145 | @Override 146 | public String asSummaryString() { 147 | return "ClickHouse table source"; 148 | } 149 | 150 | @Override 151 | public Result applyFilters(List filters) { 152 | this.filterClause = FilterPushDownHelper.convert(filters); 153 | return Result.of(new ArrayList<>(filters), new ArrayList<>(filters)); 154 | } 155 | 156 | @Override 157 | public void applyLimit(long limit) { 158 | this.limit = limit; 159 | } 160 | 161 | @Override 162 | public boolean supportsNestedProjection() { 163 | return false; 164 | } 165 | 166 | @Override 167 | public void applyProjection(int[][] projectedFields, DataType producedDataType) { 168 | this.physicalRowDataType = Projection.of(projectedFields).project(physicalRowDataType); 169 | } 170 | } 171 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/catalog/ClickHouseCatalogFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.catalog; 19 | 20 | import org.apache.flink.configuration.ConfigOption; 21 | import org.apache.flink.table.catalog.Catalog; 22 | import org.apache.flink.table.connector.source.lookup.LookupOptions; 23 | import org.apache.flink.table.factories.CatalogFactory; 24 | import org.apache.flink.table.factories.FactoryUtil; 25 | 26 | import java.util.HashSet; 27 | import java.util.Set; 28 | 29 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfig.IDENTIFIER; 30 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfig.PROPERTIES_PREFIX; 31 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.CATALOG_IGNORE_PRIMARY_KEY; 32 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.DATABASE_NAME; 33 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.PASSWORD; 34 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SCAN_PARTITION_COLUMN; 35 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SCAN_PARTITION_LOWER_BOUND; 36 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SCAN_PARTITION_NUM; 37 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SCAN_PARTITION_UPPER_BOUND; 38 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_BATCH_SIZE; 39 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_FLUSH_INTERVAL; 40 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_IGNORE_DELETE; 41 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_MAX_RETRIES; 42 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_PARALLELISM; 43 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_PARTITION_KEY; 44 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_PARTITION_STRATEGY; 45 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_SHARDING_USE_TABLE_DEF; 46 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.SINK_UPDATE_STRATEGY; 47 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.URL; 48 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.USERNAME; 49 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfigOptions.USE_LOCAL; 50 | import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; 51 | 52 | /** Factory for {@link ClickHouseCatalog}. */ 53 | public class ClickHouseCatalogFactory implements CatalogFactory { 54 | 55 | @Override 56 | public String factoryIdentifier() { 57 | return IDENTIFIER; 58 | } 59 | 60 | @Override 61 | public Set> requiredOptions() { 62 | final Set> options = new HashSet<>(); 63 | options.add(URL); 64 | options.add(USERNAME); 65 | options.add(PASSWORD); 66 | return options; 67 | } 68 | 69 | @Override 70 | public Set> optionalOptions() { 71 | final Set> options = new HashSet<>(); 72 | options.add(PROPERTY_VERSION); 73 | options.add(DATABASE_NAME); 74 | options.add(USE_LOCAL); 75 | options.add(CATALOG_IGNORE_PRIMARY_KEY); 76 | 77 | options.add(SINK_BATCH_SIZE); 78 | options.add(SINK_FLUSH_INTERVAL); 79 | options.add(SINK_MAX_RETRIES); 80 | options.add(SINK_UPDATE_STRATEGY); 81 | options.add(SINK_PARTITION_STRATEGY); 82 | options.add(SINK_PARTITION_KEY); 83 | options.add(SINK_SHARDING_USE_TABLE_DEF); 84 | options.add(SINK_IGNORE_DELETE); 85 | options.add(SINK_PARALLELISM); 86 | 87 | options.add(SCAN_PARTITION_COLUMN); 88 | options.add(SCAN_PARTITION_NUM); 89 | options.add(SCAN_PARTITION_LOWER_BOUND); 90 | options.add(SCAN_PARTITION_UPPER_BOUND); 91 | 92 | options.add(LookupOptions.CACHE_TYPE); 93 | options.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS); 94 | options.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE); 95 | options.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS); 96 | options.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY); 97 | options.add(LookupOptions.MAX_RETRIES); 98 | return options; 99 | } 100 | 101 | @Override 102 | public Catalog createCatalog(Context context) { 103 | final FactoryUtil.CatalogFactoryHelper helper = 104 | FactoryUtil.createCatalogFactoryHelper(this, context); 105 | helper.validateExcept(PROPERTIES_PREFIX); 106 | 107 | return new ClickHouseCatalog( 108 | context.getName(), 109 | helper.getOptions().get(DATABASE_NAME), 110 | helper.getOptions().get(URL), 111 | helper.getOptions().get(USERNAME), 112 | helper.getOptions().get(PASSWORD), 113 | helper.getOptions().toMap()); 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/config/ClickHouseConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.config; 19 | 20 | /** clickhouse config properties. */ 21 | public class ClickHouseConfig { 22 | 23 | public static final String IDENTIFIER = "clickhouse"; 24 | 25 | public static final String PROPERTIES_PREFIX = "properties."; 26 | 27 | public static final String URL = "url"; 28 | 29 | public static final String USERNAME = "username"; 30 | 31 | public static final String PASSWORD = "password"; 32 | 33 | public static final String DATABASE_NAME = "database-name"; 34 | 35 | public static final String TABLE_NAME = "table-name"; 36 | 37 | public static final String USE_LOCAL = "use-local"; 38 | 39 | public static final String SINK_BATCH_SIZE = "sink.batch-size"; 40 | 41 | public static final String SINK_FLUSH_INTERVAL = "sink.flush-interval"; 42 | 43 | public static final String SINK_MAX_RETRIES = "sink.max-retries"; 44 | 45 | public static final String SINK_UPDATE_STRATEGY = "sink.update-strategy"; 46 | 47 | public static final String SINK_PARTITION_STRATEGY = "sink.partition-strategy"; 48 | 49 | public static final String SINK_PARTITION_KEY = "sink.partition-key"; 50 | 51 | public static final String SINK_SHARDING_USE_TABLE_DEF = "sink.sharding.use-table-definition"; 52 | 53 | public static final String SINK_IGNORE_DELETE = "sink.ignore-delete"; 54 | 55 | public static final String CATALOG_IGNORE_PRIMARY_KEY = "catalog.ignore-primary-key"; 56 | 57 | public static final String SCAN_PARTITION_COLUMN = "scan.partition.column"; 58 | 59 | public static final String SCAN_PARTITION_NUM = "scan.partition.num"; 60 | 61 | public static final String SCAN_PARTITION_LOWER_BOUND = "scan.partition.lower-bound"; 62 | 63 | public static final String SCAN_PARTITION_UPPER_BOUND = "scan.partition.upper-bound"; 64 | } 65 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/ClickHouseBatchInputFormat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal; 19 | 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | import org.apache.flink.connector.clickhouse.internal.connection.ClickHouseConnectionProvider; 22 | import org.apache.flink.connector.clickhouse.internal.converter.ClickHouseRowConverter; 23 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseReadOptions; 24 | import org.apache.flink.core.io.InputSplit; 25 | import org.apache.flink.table.data.RowData; 26 | 27 | import com.clickhouse.jdbc.ClickHouseConnection; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.io.IOException; 32 | import java.sql.PreparedStatement; 33 | import java.sql.ResultSet; 34 | import java.sql.SQLException; 35 | 36 | /** ClickHouse batch input format. */ 37 | public class ClickHouseBatchInputFormat extends AbstractClickHouseInputFormat { 38 | 39 | private static final Logger LOG = LoggerFactory.getLogger(ClickHouseBatchInputFormat.class); 40 | 41 | private final ClickHouseConnectionProvider connectionProvider; 42 | 43 | private final ClickHouseRowConverter rowConverter; 44 | 45 | private final ClickHouseReadOptions readOptions; 46 | 47 | private transient PreparedStatement statement; 48 | private transient ResultSet resultSet; 49 | private transient boolean hasNext; 50 | 51 | public ClickHouseBatchInputFormat( 52 | ClickHouseConnectionProvider connectionProvider, 53 | ClickHouseRowConverter rowConverter, 54 | ClickHouseReadOptions readOptions, 55 | String[] fieldNames, 56 | TypeInformation rowDataTypeInfo, 57 | Object[][] parameterValues, 58 | String parameterClause, 59 | String filterClause, 60 | long limit) { 61 | super(fieldNames, rowDataTypeInfo, parameterValues, parameterClause, filterClause, limit); 62 | this.connectionProvider = connectionProvider; 63 | this.rowConverter = rowConverter; 64 | this.readOptions = readOptions; 65 | } 66 | 67 | @Override 68 | public void openInputFormat() { 69 | try { 70 | ClickHouseConnection connection = connectionProvider.getOrCreateConnection(); 71 | String query = getQuery(readOptions.getTableName(), readOptions.getDatabaseName()); 72 | statement = connection.prepareStatement(query); 73 | } catch (SQLException se) { 74 | throw new IllegalArgumentException("open() failed." + se.getMessage(), se); 75 | } 76 | } 77 | 78 | @Override 79 | public void closeInputFormat() { 80 | try { 81 | if (statement != null) { 82 | statement.close(); 83 | } 84 | } catch (SQLException exception) { 85 | LOG.info("InputFormat Statement couldn't be closed.", exception); 86 | } finally { 87 | statement = null; 88 | } 89 | 90 | if (connectionProvider != null) { 91 | connectionProvider.closeConnections(); 92 | } 93 | } 94 | 95 | @Override 96 | public void open(InputSplit split) { 97 | try { 98 | if (split != null && parameterValues != null) { 99 | for (int i = 0; i < parameterValues[split.getSplitNumber()].length; i++) { 100 | Object param = parameterValues[split.getSplitNumber()][i]; 101 | statement.setObject(i + 1, param); 102 | } 103 | } 104 | 105 | resultSet = statement.executeQuery(); 106 | hasNext = resultSet.next(); 107 | } catch (SQLException se) { 108 | throw new IllegalArgumentException("open() failed." + se.getMessage(), se); 109 | } 110 | } 111 | 112 | @Override 113 | public void close() { 114 | try { 115 | if (resultSet != null) { 116 | resultSet.close(); 117 | } 118 | } catch (SQLException se) { 119 | LOG.info("InputFormat ResultSet couldn't be closed.", se); 120 | } 121 | } 122 | 123 | @Override 124 | public boolean reachedEnd() { 125 | return !hasNext; 126 | } 127 | 128 | @Override 129 | public RowData nextRecord(RowData reuse) throws IOException { 130 | if (!hasNext) { 131 | return null; 132 | } 133 | 134 | try { 135 | RowData row = rowConverter.toInternal(resultSet); 136 | // update hasNext after we've read the record 137 | hasNext = resultSet.next(); 138 | return row; 139 | } catch (Exception exception) { 140 | throw new IOException("Couldn't read data from resultSet.", exception); 141 | } 142 | } 143 | 144 | @Override 145 | public InputSplit[] createInputSplits(int minNumSplits) { 146 | int splitNum = parameterValues != null ? parameterValues.length : 1; 147 | return createGenericInputSplits(splitNum); 148 | } 149 | } 150 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/ClickHouseBatchOutputFormat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.connection.ClickHouseConnectionProvider; 21 | import org.apache.flink.connector.clickhouse.internal.executor.ClickHouseExecutor; 22 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseDmlOptions; 23 | import org.apache.flink.table.data.RowData; 24 | import org.apache.flink.table.types.logical.LogicalType; 25 | import org.apache.flink.util.Preconditions; 26 | 27 | import javax.annotation.Nonnull; 28 | 29 | import java.io.IOException; 30 | import java.sql.SQLException; 31 | 32 | /** Output data to ClickHouse local table. */ 33 | public class ClickHouseBatchOutputFormat extends AbstractClickHouseOutputFormat { 34 | 35 | private static final long serialVersionUID = 1L; 36 | 37 | private final ClickHouseConnectionProvider connectionProvider; 38 | 39 | private final String[] fieldNames; 40 | 41 | private final String[] keyFields; 42 | 43 | private final String[] partitionFields; 44 | 45 | private final LogicalType[] fieldTypes; 46 | 47 | private final ClickHouseDmlOptions options; 48 | 49 | private transient ClickHouseExecutor executor; 50 | 51 | private transient int batchCount = 0; 52 | 53 | protected ClickHouseBatchOutputFormat( 54 | @Nonnull ClickHouseConnectionProvider connectionProvider, 55 | @Nonnull String[] fieldNames, 56 | @Nonnull String[] keyFields, 57 | @Nonnull String[] partitionFields, 58 | @Nonnull LogicalType[] fieldTypes, 59 | @Nonnull ClickHouseDmlOptions options) { 60 | this.connectionProvider = Preconditions.checkNotNull(connectionProvider); 61 | this.fieldNames = Preconditions.checkNotNull(fieldNames); 62 | this.keyFields = Preconditions.checkNotNull(keyFields); 63 | this.partitionFields = Preconditions.checkNotNull(partitionFields); 64 | this.fieldTypes = Preconditions.checkNotNull(fieldTypes); 65 | this.options = Preconditions.checkNotNull(options); 66 | } 67 | 68 | @Override 69 | public void open(int taskNumber, int numTasks) throws IOException { 70 | try { 71 | // TODO Distributed tables don't support update and delete statements. 72 | executor = 73 | ClickHouseExecutor.createClickHouseExecutor( 74 | options.getTableName(), 75 | options.getDatabaseName(), 76 | null, 77 | fieldNames, 78 | keyFields, 79 | partitionFields, 80 | fieldTypes, 81 | options); 82 | executor.prepareStatement(connectionProvider); 83 | executor.setRuntimeContext(getRuntimeContext()); 84 | 85 | long flushIntervalMillis = options.getFlushInterval().toMillis(); 86 | scheduledFlush(flushIntervalMillis, "clickhouse-batch-output-format"); 87 | } catch (Exception exception) { 88 | throw new IOException("Unable to establish connection with ClickHouse.", exception); 89 | } 90 | } 91 | 92 | @Override 93 | public synchronized void writeRecord(RowData record) throws IOException { 94 | checkFlushException(); 95 | 96 | try { 97 | executor.addToBatch(record); 98 | batchCount++; 99 | if (batchCount >= options.getBatchSize()) { 100 | flush(); 101 | } 102 | } catch (SQLException exception) { 103 | throw new IOException("Writing record to ClickHouse statement failed.", exception); 104 | } 105 | } 106 | 107 | @Override 108 | public synchronized void flush() throws IOException { 109 | if (batchCount > 0) { 110 | checkBeforeFlush(executor); 111 | batchCount = 0; 112 | } 113 | } 114 | 115 | @Override 116 | public synchronized void closeOutputFormat() { 117 | executor.closeStatement(); 118 | connectionProvider.closeConnections(); 119 | } 120 | } 121 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/ClickHouseRowDataSinkFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.api.common.functions.RuntimeContext; 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.runtime.state.FunctionInitializationContext; 24 | import org.apache.flink.runtime.state.FunctionSnapshotContext; 25 | import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; 26 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; 27 | import org.apache.flink.table.data.RowData; 28 | import org.apache.flink.util.Preconditions; 29 | 30 | import javax.annotation.Nonnull; 31 | 32 | import java.io.IOException; 33 | 34 | /** A rich sink function to write {@link RowData} records into ClickHouse. */ 35 | @Internal 36 | public class ClickHouseRowDataSinkFunction extends RichSinkFunction 37 | implements CheckpointedFunction { 38 | 39 | private final AbstractClickHouseOutputFormat outputFormat; 40 | 41 | public ClickHouseRowDataSinkFunction(@Nonnull AbstractClickHouseOutputFormat outputFormat) { 42 | this.outputFormat = Preconditions.checkNotNull(outputFormat); 43 | } 44 | 45 | @Override 46 | public void open(Configuration parameters) throws Exception { 47 | outputFormat.configure(parameters); 48 | RuntimeContext runtimeContext = getRuntimeContext(); 49 | outputFormat.setRuntimeContext(runtimeContext); 50 | outputFormat.open( 51 | runtimeContext.getIndexOfThisSubtask(), 52 | runtimeContext.getNumberOfParallelSubtasks()); 53 | } 54 | 55 | @Override 56 | public void invoke(RowData value, Context context) throws IOException { 57 | outputFormat.writeRecord(value); 58 | } 59 | 60 | @Override 61 | public void initializeState(FunctionInitializationContext context) {} 62 | 63 | @Override 64 | public void snapshotState(FunctionSnapshotContext context) throws Exception { 65 | outputFormat.flush(); 66 | } 67 | 68 | @Override 69 | public void close() { 70 | outputFormat.close(); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/ClickHouseShardOutputFormat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.connection.ClickHouseConnectionProvider; 21 | import org.apache.flink.connector.clickhouse.internal.executor.ClickHouseExecutor; 22 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseDmlOptions; 23 | import org.apache.flink.connector.clickhouse.internal.partitioner.ClickHousePartitioner; 24 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 25 | import org.apache.flink.connector.clickhouse.internal.schema.DistributedEngineFull; 26 | import org.apache.flink.table.data.RowData; 27 | import org.apache.flink.table.types.logical.LogicalType; 28 | import org.apache.flink.util.Preconditions; 29 | 30 | import com.clickhouse.jdbc.ClickHouseConnection; 31 | 32 | import javax.annotation.Nonnull; 33 | 34 | import java.io.IOException; 35 | import java.util.HashMap; 36 | import java.util.Map; 37 | import java.util.concurrent.atomic.AtomicInteger; 38 | 39 | /** The shard output format of distributed table. */ 40 | public class ClickHouseShardOutputFormat extends AbstractClickHouseOutputFormat { 41 | 42 | private static final long serialVersionUID = 1L; 43 | 44 | private final ClickHouseConnectionProvider connectionProvider; 45 | 46 | private final ClusterSpec clusterSpec; 47 | 48 | private final DistributedEngineFull shardTableSchema; 49 | 50 | private final String[] fieldNames; 51 | 52 | private final String[] keyFields; 53 | 54 | private final String[] partitionFields; 55 | 56 | private final LogicalType[] logicalTypes; 57 | 58 | private final ClickHousePartitioner partitioner; 59 | 60 | private final ClickHouseDmlOptions options; 61 | 62 | private final Map shardExecutorMap; 63 | 64 | private final Map batchCountMap; 65 | 66 | protected ClickHouseShardOutputFormat( 67 | @Nonnull ClickHouseConnectionProvider connectionProvider, 68 | @Nonnull ClusterSpec clusterSpec, 69 | @Nonnull DistributedEngineFull shardTableSchema, 70 | @Nonnull String[] fieldNames, 71 | @Nonnull String[] keyFields, 72 | @Nonnull String[] partitionFields, 73 | @Nonnull LogicalType[] logicalTypes, 74 | @Nonnull ClickHousePartitioner partitioner, 75 | @Nonnull ClickHouseDmlOptions options) { 76 | this.connectionProvider = Preconditions.checkNotNull(connectionProvider); 77 | this.clusterSpec = Preconditions.checkNotNull(clusterSpec); 78 | this.shardTableSchema = Preconditions.checkNotNull(shardTableSchema); 79 | this.fieldNames = Preconditions.checkNotNull(fieldNames); 80 | this.keyFields = keyFields; 81 | this.partitionFields = partitionFields; 82 | this.logicalTypes = Preconditions.checkNotNull(logicalTypes); 83 | this.partitioner = Preconditions.checkNotNull(partitioner); 84 | this.options = Preconditions.checkNotNull(options); 85 | this.shardExecutorMap = new HashMap<>(); 86 | this.batchCountMap = new HashMap<>(); 87 | } 88 | 89 | @Override 90 | public void open(int taskNumber, int numTasks) throws IOException { 91 | try { 92 | Map connectionMap = 93 | connectionProvider.createShardConnections( 94 | clusterSpec, shardTableSchema.getDatabase()); 95 | for (Map.Entry connectionEntry : 96 | connectionMap.entrySet()) { 97 | ClickHouseExecutor executor = 98 | ClickHouseExecutor.createClickHouseExecutor( 99 | shardTableSchema.getTable(), 100 | shardTableSchema.getDatabase(), 101 | shardTableSchema.getCluster(), 102 | fieldNames, 103 | keyFields, 104 | partitionFields, 105 | logicalTypes, 106 | options); 107 | executor.prepareStatement(connectionEntry.getValue()); 108 | shardExecutorMap.put(connectionEntry.getKey(), executor); 109 | } 110 | 111 | long flushIntervalMillis = options.getFlushInterval().toMillis(); 112 | scheduledFlush(flushIntervalMillis, "clickhouse-shard-output-format"); 113 | } catch (Exception exception) { 114 | throw new IOException("Unable to establish connection to ClickHouse", exception); 115 | } 116 | } 117 | 118 | @Override 119 | public synchronized void writeRecord(RowData record) throws IOException { 120 | checkFlushException(); 121 | 122 | switch (record.getRowKind()) { 123 | case INSERT: 124 | case UPDATE_AFTER: 125 | case DELETE: 126 | writeRecordToOneExecutor(record); 127 | break; 128 | case UPDATE_BEFORE: 129 | break; 130 | default: 131 | throw new UnsupportedOperationException( 132 | String.format( 133 | "Unknown row kind, the supported row kinds is: INSERT, UPDATE_BEFORE, UPDATE_AFTER, DELETE, but get: %s.", 134 | record.getRowKind())); 135 | } 136 | } 137 | 138 | private void writeRecordToOneExecutor(RowData record) throws IOException { 139 | try { 140 | int shardNum = partitioner.select(record, clusterSpec); 141 | shardExecutorMap.get(shardNum).addToBatch(record); 142 | int batchCount = 143 | batchCountMap 144 | .computeIfAbsent(shardNum, integer -> new AtomicInteger(0)) 145 | .incrementAndGet(); 146 | if (batchCount >= options.getBatchSize()) { 147 | flush(shardNum); 148 | } 149 | } catch (Exception exception) { 150 | throw new IOException("Writing record to one executor failed.", exception); 151 | } 152 | } 153 | 154 | @Override 155 | public synchronized void flush() throws IOException { 156 | for (Integer shardNum : shardExecutorMap.keySet()) { 157 | flush(shardNum); 158 | } 159 | } 160 | 161 | private synchronized void flush(int shardNum) throws IOException { 162 | AtomicInteger batchCount = batchCountMap.get(shardNum); 163 | if (batchCount != null && batchCount.intValue() > 0) { 164 | checkBeforeFlush(shardExecutorMap.get(shardNum)); 165 | batchCount.set(0); 166 | } 167 | } 168 | 169 | @Override 170 | public synchronized void closeOutputFormat() { 171 | for (ClickHouseExecutor shardExecutor : shardExecutorMap.values()) { 172 | shardExecutor.closeStatement(); 173 | } 174 | connectionProvider.closeConnections(); 175 | } 176 | } 177 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/ClickHouseStatementFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal; 19 | 20 | import org.apache.flink.connector.clickhouse.util.ClickHouseUtil; 21 | 22 | import org.apache.commons.lang3.ArrayUtils; 23 | 24 | import java.util.Arrays; 25 | import java.util.stream.Collectors; 26 | 27 | import static java.lang.String.format; 28 | import static java.util.stream.Collectors.joining; 29 | import static org.apache.flink.connector.clickhouse.util.ClickHouseUtil.quoteIdentifier; 30 | 31 | /** Create an insert/update/delete ClickHouse statement. */ 32 | public class ClickHouseStatementFactory { 33 | 34 | private static final String EMPTY = ""; 35 | 36 | private ClickHouseStatementFactory() {} 37 | 38 | public static String getSelectStatement( 39 | String tableName, String databaseName, String[] selectFields) { 40 | String selectClause = 41 | Arrays.stream(selectFields) 42 | .map(ClickHouseUtil::quoteIdentifier) 43 | .collect(joining(", ")); 44 | return String.join( 45 | EMPTY, "SELECT ", selectClause, " FROM ", fromTableClause(tableName, databaseName)); 46 | } 47 | 48 | public static String getSelectWhereStatement( 49 | String tableName, 50 | String databaseName, 51 | String[] selectFields, 52 | String[] conditionFields) { 53 | String selectStatement = getSelectStatement(tableName, databaseName, selectFields); 54 | String whereClause = 55 | Arrays.stream(conditionFields) 56 | .map(f -> format("%s = ?", quoteIdentifier(f))) 57 | .collect(Collectors.joining(" AND ")); 58 | return selectStatement + (conditionFields.length > 0 ? " WHERE " + whereClause : ""); 59 | } 60 | 61 | public static String getInsertIntoStatement( 62 | String tableName, String databaseName, String[] fieldNames) { 63 | String columns = 64 | Arrays.stream(fieldNames) 65 | .map(ClickHouseUtil::quoteIdentifier) 66 | .collect(joining(", ")); 67 | String placeholders = Arrays.stream(fieldNames).map((f) -> "?").collect(joining(", ")); 68 | return String.join( 69 | EMPTY, 70 | "INSERT INTO ", 71 | fromTableClause(tableName, databaseName), 72 | "(", 73 | columns, 74 | ") VALUES (", 75 | placeholders, 76 | ")"); 77 | } 78 | 79 | public static String getUpdateStatement( 80 | String tableName, 81 | String databaseName, 82 | String clusterName, 83 | String[] fieldNames, 84 | String[] keyFields, 85 | String[] partitionFields) { 86 | String setClause = 87 | Arrays.stream(fieldNames) 88 | .filter(f -> !ArrayUtils.contains(keyFields, f)) 89 | .filter(f -> !ArrayUtils.contains(partitionFields, f)) 90 | .map((f) -> quoteIdentifier(f) + "=?") 91 | .collect(joining(", ")); 92 | String conditionClause = 93 | Arrays.stream(keyFields) 94 | .map((f) -> quoteIdentifier(f) + "=?") 95 | .collect(joining(" AND ")); 96 | String onClusterClause = ""; 97 | if (clusterName != null) { 98 | onClusterClause = " ON CLUSTER " + quoteIdentifier(clusterName); 99 | } 100 | 101 | return String.join( 102 | EMPTY, 103 | "ALTER TABLE ", 104 | fromTableClause(tableName, databaseName), 105 | onClusterClause, 106 | " UPDATE ", 107 | setClause, 108 | " WHERE ", 109 | conditionClause); 110 | } 111 | 112 | public static String getDeleteStatement( 113 | String tableName, String databaseName, String clusterName, String[] conditionFields) { 114 | String conditionClause = 115 | Arrays.stream(conditionFields) 116 | .map((f) -> quoteIdentifier(f) + "=?") 117 | .collect(joining(" AND ")); 118 | String onClusterClause = ""; 119 | if (clusterName != null) { 120 | onClusterClause = " ON CLUSTER " + quoteIdentifier(clusterName); 121 | } 122 | 123 | return String.join( 124 | EMPTY, 125 | "ALTER TABLE ", 126 | fromTableClause(tableName, databaseName), 127 | onClusterClause, 128 | " DELETE WHERE ", 129 | conditionClause); 130 | } 131 | 132 | private static String fromTableClause(String tableName, String databaseName) { 133 | if (databaseName == null) { 134 | return quoteIdentifier(tableName); 135 | } 136 | 137 | return format("%s.%s", quoteIdentifier(databaseName), quoteIdentifier(tableName)); 138 | } 139 | } 140 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/connection/ClickHouseConnectionProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.connection; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseConnectionOptions; 21 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 22 | import org.apache.flink.connector.clickhouse.internal.schema.ShardSpec; 23 | 24 | import com.clickhouse.client.config.ClickHouseDefaults; 25 | import com.clickhouse.jdbc.ClickHouseConnection; 26 | import com.clickhouse.jdbc.ClickHouseDriver; 27 | import org.slf4j.Logger; 28 | import org.slf4j.LoggerFactory; 29 | 30 | import java.io.Serializable; 31 | import java.sql.SQLException; 32 | import java.util.ArrayList; 33 | import java.util.HashMap; 34 | import java.util.List; 35 | import java.util.Map; 36 | import java.util.Properties; 37 | 38 | import static java.util.stream.Collectors.toList; 39 | import static org.apache.flink.connector.clickhouse.util.ClickHouseJdbcUtil.getClusterSpec; 40 | 41 | /** ClickHouse connection provider. Use ClickHouseDriver to create a connection. */ 42 | public class ClickHouseConnectionProvider implements Serializable { 43 | 44 | private static final long serialVersionUID = 1L; 45 | 46 | private static final Logger LOG = LoggerFactory.getLogger(ClickHouseConnectionProvider.class); 47 | 48 | private final ClickHouseConnectionOptions options; 49 | 50 | private final Properties connectionProperties; 51 | 52 | private transient ClickHouseConnection connection; 53 | 54 | private transient List shardConnections; 55 | 56 | public ClickHouseConnectionProvider(ClickHouseConnectionOptions options) { 57 | this(options, new Properties()); 58 | } 59 | 60 | public ClickHouseConnectionProvider( 61 | ClickHouseConnectionOptions options, Properties connectionProperties) { 62 | this.options = options; 63 | this.connectionProperties = connectionProperties; 64 | } 65 | 66 | public boolean isConnectionValid() throws SQLException { 67 | return connection != null; 68 | } 69 | 70 | public synchronized ClickHouseConnection getOrCreateConnection() throws SQLException { 71 | if (connection == null) { 72 | connection = createConnection(options.getUrl()); 73 | } 74 | return connection; 75 | } 76 | 77 | public synchronized Map createShardConnections( 78 | ClusterSpec clusterSpec, String defaultDatabase) throws SQLException { 79 | Map connectionMap = new HashMap<>(); 80 | String urlSuffix = options.getUrlSuffix(); 81 | for (ShardSpec shardSpec : clusterSpec.getShards()) { 82 | String shardUrl = shardSpec.getJdbcUrls() + urlSuffix; 83 | ClickHouseConnection connection = 84 | createAndStoreShardConnection(shardUrl, defaultDatabase); 85 | connectionMap.put(shardSpec.getNum(), connection); 86 | } 87 | 88 | return connectionMap; 89 | } 90 | 91 | public synchronized ClickHouseConnection createAndStoreShardConnection( 92 | String url, String database) throws SQLException { 93 | if (shardConnections == null) { 94 | shardConnections = new ArrayList<>(); 95 | } 96 | 97 | ClickHouseConnection connection = createConnection(url); 98 | shardConnections.add(connection); 99 | return connection; 100 | } 101 | 102 | public List getShardUrls(String remoteCluster) throws SQLException { 103 | Map shardsMap = new HashMap<>(); 104 | ClickHouseConnection conn = getOrCreateConnection(); 105 | ClusterSpec clusterSpec = getClusterSpec(conn, remoteCluster); 106 | String urlSuffix = options.getUrlSuffix(); 107 | for (ShardSpec shardSpec : clusterSpec.getShards()) { 108 | String shardUrl = shardSpec.getJdbcUrls() + urlSuffix; 109 | shardsMap.put(shardSpec.getNum(), shardUrl); 110 | } 111 | 112 | return shardsMap.entrySet().stream() 113 | .sorted(Map.Entry.comparingByKey()) 114 | .map(Map.Entry::getValue) 115 | .collect(toList()); 116 | } 117 | 118 | private ClickHouseConnection createConnection(String url) throws SQLException { 119 | LOG.info("connecting to {}", url); 120 | Properties configuration = new Properties(); 121 | configuration.putAll(connectionProperties); 122 | if (options.getUsername().isPresent()) { 123 | configuration.setProperty( 124 | ClickHouseDefaults.USER.getKey(), options.getUsername().get()); 125 | } 126 | if (options.getPassword().isPresent()) { 127 | configuration.setProperty( 128 | ClickHouseDefaults.PASSWORD.getKey(), options.getPassword().get()); 129 | } 130 | ClickHouseDriver driver = new ClickHouseDriver(); 131 | return driver.connect(url, configuration); 132 | } 133 | 134 | public void closeConnections() { 135 | if (this.connection != null) { 136 | try { 137 | connection.close(); 138 | } catch (SQLException exception) { 139 | LOG.warn("ClickHouse connection could not be closed.", exception); 140 | } finally { 141 | connection = null; 142 | } 143 | } 144 | 145 | if (shardConnections != null) { 146 | for (ClickHouseConnection shardConnection : this.shardConnections) { 147 | try { 148 | shardConnection.close(); 149 | } catch (SQLException exception) { 150 | LOG.warn("ClickHouse shard connection could not be closed.", exception); 151 | } 152 | } 153 | 154 | shardConnections = null; 155 | } 156 | } 157 | } 158 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/connection/ClickHouseStatementWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.connection; 19 | 20 | import com.clickhouse.jdbc.ClickHousePreparedStatement; 21 | 22 | import java.math.BigDecimal; 23 | import java.sql.Date; 24 | import java.sql.ResultSet; 25 | import java.sql.SQLException; 26 | import java.sql.Timestamp; 27 | 28 | /** Wrapper class for ClickHousePreparedStatement. */ 29 | public class ClickHouseStatementWrapper { 30 | public final ClickHousePreparedStatement statement; 31 | 32 | public ClickHouseStatementWrapper(ClickHousePreparedStatement statement) { 33 | this.statement = statement; 34 | } 35 | 36 | public void addBatch() throws SQLException { 37 | statement.addBatch(); 38 | } 39 | 40 | public int[] executeBatch() throws SQLException { 41 | return statement.executeBatch(); 42 | } 43 | 44 | public void close() throws SQLException { 45 | statement.close(); 46 | } 47 | 48 | public void setBoolean(int parameterIndex, boolean x) throws SQLException { 49 | statement.setBoolean(parameterIndex, x); 50 | } 51 | 52 | public void setByte(int parameterIndex, byte x) throws SQLException { 53 | statement.setByte(parameterIndex, x); 54 | } 55 | 56 | public void setShort(int parameterIndex, short x) throws SQLException { 57 | statement.setShort(parameterIndex, x); 58 | } 59 | 60 | public void setInt(int parameterIndex, int x) throws SQLException { 61 | statement.setInt(parameterIndex, x); 62 | } 63 | 64 | public void setLong(int parameterIndex, long x) throws SQLException { 65 | statement.setLong(parameterIndex, x); 66 | } 67 | 68 | public void setFloat(int parameterIndex, float x) throws SQLException { 69 | statement.setFloat(parameterIndex, x); 70 | } 71 | 72 | public void setDouble(int parameterIndex, double x) throws SQLException { 73 | statement.setDouble(parameterIndex, x); 74 | } 75 | 76 | public void setBigDecimal(int parameterIndex, BigDecimal x) throws SQLException { 77 | statement.setBigDecimal(parameterIndex, x); 78 | } 79 | 80 | public void setString(int parameterIndex, String x) throws SQLException { 81 | statement.setString(parameterIndex, x); 82 | } 83 | 84 | public void setBytes(int parameterIndex, byte[] x) throws SQLException { 85 | statement.setBytes(parameterIndex, x); 86 | } 87 | 88 | public void setDate(int parameterIndex, Date x) throws SQLException { 89 | statement.setDate(parameterIndex, x); 90 | } 91 | 92 | public void setTimestamp(int parameterIndex, Timestamp x) throws SQLException { 93 | statement.setTimestamp(parameterIndex, x); 94 | } 95 | 96 | public void setArray(int parameterIndex, Object[] array) throws SQLException { 97 | statement.setArray(parameterIndex, new ObjectArray(array)); 98 | } 99 | 100 | public void setObject(int parameterIndex, Object x) throws SQLException { 101 | statement.setObject(parameterIndex, x); 102 | } 103 | 104 | public void clearParameters() throws SQLException { 105 | statement.clearParameters(); 106 | } 107 | 108 | public ResultSet executeQuery() throws SQLException { 109 | return statement.executeQuery(); 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/connection/ObjectArray.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.connection; 19 | 20 | import org.apache.flink.util.Preconditions; 21 | 22 | import java.sql.Array; 23 | import java.sql.ResultSet; 24 | import java.util.Map; 25 | 26 | /** Wrap object array. */ 27 | public class ObjectArray implements Array { 28 | 29 | private Object[] array; 30 | 31 | public ObjectArray(Object[] array) { 32 | this.array = Preconditions.checkNotNull(array); 33 | } 34 | 35 | @Override 36 | public String getBaseTypeName() { 37 | throw new UnsupportedOperationException(); 38 | } 39 | 40 | @Override 41 | public int getBaseType() { 42 | throw new UnsupportedOperationException(); 43 | } 44 | 45 | @Override 46 | public Object getArray() { 47 | return array; 48 | } 49 | 50 | @Override 51 | public Object getArray(Map> map) { 52 | throw new UnsupportedOperationException(); 53 | } 54 | 55 | @Override 56 | public Object getArray(long index, int count) { 57 | throw new UnsupportedOperationException(); 58 | } 59 | 60 | @Override 61 | public Object getArray(long index, int count, Map> map) { 62 | throw new UnsupportedOperationException(); 63 | } 64 | 65 | @Override 66 | public ResultSet getResultSet() { 67 | throw new UnsupportedOperationException(); 68 | } 69 | 70 | @Override 71 | public ResultSet getResultSet(Map> map) { 72 | throw new UnsupportedOperationException(); 73 | } 74 | 75 | @Override 76 | public ResultSet getResultSet(long index, int count) { 77 | throw new UnsupportedOperationException(); 78 | } 79 | 80 | @Override 81 | public ResultSet getResultSet(long index, int count, Map> map) { 82 | throw new UnsupportedOperationException(); 83 | } 84 | 85 | @Override 86 | public void free() { 87 | this.array = null; 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/executor/ClickHouseBatchExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.executor; 19 | 20 | import org.apache.flink.api.common.functions.RuntimeContext; 21 | import org.apache.flink.connector.clickhouse.internal.connection.ClickHouseConnectionProvider; 22 | import org.apache.flink.connector.clickhouse.internal.connection.ClickHouseStatementWrapper; 23 | import org.apache.flink.connector.clickhouse.internal.converter.ClickHouseRowConverter; 24 | import org.apache.flink.connector.clickhouse.internal.options.ClickHouseDmlOptions; 25 | import org.apache.flink.table.data.RowData; 26 | 27 | import com.clickhouse.jdbc.ClickHouseConnection; 28 | import com.clickhouse.jdbc.ClickHousePreparedStatement; 29 | import org.slf4j.Logger; 30 | import org.slf4j.LoggerFactory; 31 | 32 | import java.sql.SQLException; 33 | 34 | /** ClickHouse's batch executor. */ 35 | public class ClickHouseBatchExecutor implements ClickHouseExecutor { 36 | 37 | private static final long serialVersionUID = 1L; 38 | 39 | private static final Logger LOG = LoggerFactory.getLogger(ClickHouseBatchExecutor.class); 40 | 41 | private final String insertSql; 42 | 43 | private final ClickHouseRowConverter converter; 44 | 45 | private final int maxRetries; 46 | 47 | private transient ClickHouseStatementWrapper statement; 48 | 49 | private transient ClickHouseConnectionProvider connectionProvider; 50 | 51 | public ClickHouseBatchExecutor( 52 | String insertSql, ClickHouseRowConverter converter, ClickHouseDmlOptions options) { 53 | this.insertSql = insertSql; 54 | this.converter = converter; 55 | this.maxRetries = options.getMaxRetries(); 56 | } 57 | 58 | @Override 59 | public void prepareStatement(ClickHouseConnection connection) throws SQLException { 60 | statement = 61 | new ClickHouseStatementWrapper( 62 | (ClickHousePreparedStatement) connection.prepareStatement(insertSql)); 63 | } 64 | 65 | @Override 66 | public void prepareStatement(ClickHouseConnectionProvider connectionProvider) 67 | throws SQLException { 68 | this.connectionProvider = connectionProvider; 69 | prepareStatement(connectionProvider.getOrCreateConnection()); 70 | } 71 | 72 | @Override 73 | public void setRuntimeContext(RuntimeContext context) {} 74 | 75 | @Override 76 | public void addToBatch(RowData record) throws SQLException { 77 | switch (record.getRowKind()) { 78 | case INSERT: 79 | converter.toExternal(record, statement); 80 | statement.addBatch(); 81 | break; 82 | case UPDATE_AFTER: 83 | case DELETE: 84 | case UPDATE_BEFORE: 85 | break; 86 | default: 87 | throw new UnsupportedOperationException( 88 | String.format( 89 | "Unknown row kind, the supported row kinds is: INSERT, UPDATE_BEFORE, UPDATE_AFTER, DELETE, but get: %s.", 90 | record.getRowKind())); 91 | } 92 | } 93 | 94 | @Override 95 | public void executeBatch() throws SQLException { 96 | attemptExecuteBatch(statement, maxRetries); 97 | } 98 | 99 | @Override 100 | public void closeStatement() { 101 | if (statement != null) { 102 | try { 103 | statement.close(); 104 | } catch (SQLException exception) { 105 | LOG.warn("ClickHouse batch statement could not be closed.", exception); 106 | } finally { 107 | statement = null; 108 | } 109 | } 110 | } 111 | 112 | @Override 113 | public String toString() { 114 | return "ClickHouseBatchExecutor{" 115 | + "insertSql='" 116 | + insertSql 117 | + '\'' 118 | + ", maxRetries=" 119 | + maxRetries 120 | + ", connectionProvider=" 121 | + connectionProvider 122 | + '}'; 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/options/ClickHouseConnectionOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.options; 19 | 20 | import org.apache.flink.annotation.VisibleForTesting; 21 | 22 | import javax.annotation.Nullable; 23 | 24 | import java.io.Serializable; 25 | import java.util.Optional; 26 | import java.util.regex.Matcher; 27 | import java.util.regex.Pattern; 28 | 29 | import static org.apache.flink.connector.clickhouse.util.ClickHouseUtil.EMPTY; 30 | import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; 31 | 32 | /** ClickHouse connection options. */ 33 | public class ClickHouseConnectionOptions implements Serializable { 34 | 35 | private static final long serialVersionUID = 1L; 36 | 37 | public static final Pattern URL_PATTERN = 38 | Pattern.compile("[^/]+//[^/?]+(/(?[^?]*))?(\\?(?\\S+))?"); 39 | 40 | private final String url; 41 | 42 | private final String username; 43 | 44 | private final String password; 45 | 46 | private final String databaseName; 47 | 48 | private final String tableName; 49 | 50 | // For testing. 51 | @VisibleForTesting 52 | public ClickHouseConnectionOptions(String url) { 53 | this(url, null, null, null, null); 54 | } 55 | 56 | protected ClickHouseConnectionOptions( 57 | String url, 58 | @Nullable String username, 59 | @Nullable String password, 60 | String databaseName, 61 | String tableName) { 62 | this.url = url; 63 | this.username = username; 64 | this.password = password; 65 | this.databaseName = databaseName; 66 | this.tableName = tableName; 67 | } 68 | 69 | /** 70 | * The format of the URL suffix is as follows: {@code 71 | * [/][?param1=value1¶m2=value2]}. 72 | */ 73 | public String getUrlSuffix() { 74 | Matcher matcher = URL_PATTERN.matcher(url); 75 | if (!matcher.find()) { 76 | return EMPTY; 77 | } 78 | 79 | String database = matcher.group("database"); 80 | String param = matcher.group("param"); 81 | database = isNullOrWhitespaceOnly(database) ? EMPTY : "/" + database; 82 | param = isNullOrWhitespaceOnly(param) ? EMPTY : "?" + param; 83 | return database + param; 84 | } 85 | 86 | public String getUrl() { 87 | return this.url; 88 | } 89 | 90 | public Optional getUsername() { 91 | return Optional.ofNullable(this.username); 92 | } 93 | 94 | public Optional getPassword() { 95 | return Optional.ofNullable(this.password); 96 | } 97 | 98 | public String getDatabaseName() { 99 | return this.databaseName; 100 | } 101 | 102 | public String getTableName() { 103 | return this.tableName; 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/options/ClickHouseReadOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.options; 19 | 20 | import javax.annotation.Nullable; 21 | 22 | /** ClickHouse read options. */ 23 | public class ClickHouseReadOptions extends ClickHouseConnectionOptions { 24 | 25 | private static final long serialVersionUID = 1L; 26 | 27 | private final boolean useLocal; 28 | 29 | private final String partitionColumn; 30 | private final Integer partitionNum; 31 | private final Long partitionLowerBound; 32 | private final Long partitionUpperBound; 33 | 34 | private ClickHouseReadOptions( 35 | String url, 36 | @Nullable String username, 37 | @Nullable String password, 38 | String databaseName, 39 | String tableName, 40 | boolean useLocal, 41 | String partitionColumn, 42 | Integer partitionNum, 43 | Long partitionLowerBound, 44 | Long partitionUpperBound) { 45 | super(url, username, password, databaseName, tableName); 46 | this.useLocal = useLocal; 47 | this.partitionColumn = partitionColumn; 48 | this.partitionNum = partitionNum; 49 | this.partitionLowerBound = partitionLowerBound; 50 | this.partitionUpperBound = partitionUpperBound; 51 | } 52 | 53 | public boolean isUseLocal() { 54 | return useLocal; 55 | } 56 | 57 | public String getPartitionColumn() { 58 | return partitionColumn; 59 | } 60 | 61 | public Integer getPartitionNum() { 62 | return partitionNum; 63 | } 64 | 65 | public Long getPartitionLowerBound() { 66 | return partitionLowerBound; 67 | } 68 | 69 | public Long getPartitionUpperBound() { 70 | return partitionUpperBound; 71 | } 72 | 73 | /** Builder for {@link ClickHouseReadOptions}. */ 74 | public static class Builder { 75 | private String url; 76 | private String username; 77 | private String password; 78 | private String databaseName; 79 | private String tableName; 80 | private boolean useLocal; 81 | private String partitionColumn; 82 | private Integer partitionNum; 83 | private Long partitionLowerBound; 84 | private Long partitionUpperBound; 85 | 86 | public Builder withUrl(String url) { 87 | this.url = url; 88 | return this; 89 | } 90 | 91 | public Builder withUsername(String username) { 92 | this.username = username; 93 | return this; 94 | } 95 | 96 | public Builder withPassword(String password) { 97 | this.password = password; 98 | return this; 99 | } 100 | 101 | public Builder withDatabaseName(String databaseName) { 102 | this.databaseName = databaseName; 103 | return this; 104 | } 105 | 106 | public Builder withTableName(String tableName) { 107 | this.tableName = tableName; 108 | return this; 109 | } 110 | 111 | public Builder withUseLocal(boolean useLocal) { 112 | this.useLocal = useLocal; 113 | return this; 114 | } 115 | 116 | public Builder withPartitionColumn(String partitionColumn) { 117 | this.partitionColumn = partitionColumn; 118 | return this; 119 | } 120 | 121 | public Builder withPartitionNum(Integer partitionNum) { 122 | this.partitionNum = partitionNum; 123 | return this; 124 | } 125 | 126 | public Builder withPartitionLowerBound(Long partitionLowerBound) { 127 | this.partitionLowerBound = partitionLowerBound; 128 | return this; 129 | } 130 | 131 | public Builder withPartitionUpperBound(Long partitionUpperBound) { 132 | this.partitionUpperBound = partitionUpperBound; 133 | return this; 134 | } 135 | 136 | public ClickHouseReadOptions build() { 137 | return new ClickHouseReadOptions( 138 | url, 139 | username, 140 | password, 141 | databaseName, 142 | tableName, 143 | useLocal, 144 | partitionColumn, 145 | partitionNum, 146 | partitionLowerBound, 147 | partitionUpperBound); 148 | } 149 | } 150 | } 151 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/partitioner/BalancedPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.partitioner; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 21 | import org.apache.flink.connector.clickhouse.internal.schema.ShardSpec; 22 | import org.apache.flink.table.data.RowData; 23 | 24 | import java.util.List; 25 | 26 | /** Use round-robin mode to partition data. */ 27 | public class BalancedPartitioner extends ClickHousePartitioner { 28 | 29 | private static final long serialVersionUID = 1L; 30 | 31 | private int nextShard = 0; 32 | 33 | public BalancedPartitioner() {} 34 | 35 | @Override 36 | public int select(RowData record, ClusterSpec clusterSpec) { 37 | List shards = clusterSpec.getShards(); 38 | nextShard = (nextShard + 1) % shards.size(); 39 | return shards.get(nextShard).getNum(); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/partitioner/ClickHousePartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.partitioner; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 21 | import org.apache.flink.connector.clickhouse.internal.schema.ShardSpec; 22 | import org.apache.flink.table.data.RowData; 23 | 24 | import java.io.Serializable; 25 | 26 | /** ClickHouse data partitioner interface. */ 27 | public abstract class ClickHousePartitioner implements Serializable { 28 | 29 | private static final long serialVersionUID = 1L; 30 | 31 | public abstract int select(RowData record, ClusterSpec clusterSpec); 32 | 33 | public int select(long value, ClusterSpec clusterSpec) { 34 | value = Math.abs(value); 35 | for (ShardSpec shard : clusterSpec.getShards()) { 36 | if (shard.isInShardRangeBounds(value)) { 37 | return shard.getNum(); 38 | } 39 | } 40 | 41 | throw new IllegalStateException( 42 | String.format( 43 | "Unreachable, partitioner: %s must has some kind of bug", 44 | this.getClass().getName())); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/partitioner/JavaHashPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.partitioner; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 21 | import org.apache.flink.table.data.RowData; 22 | import org.apache.flink.table.data.RowData.FieldGetter; 23 | 24 | import java.util.List; 25 | import java.util.Objects; 26 | 27 | import static java.util.Objects.nonNull; 28 | import static org.apache.flink.util.Preconditions.checkArgument; 29 | 30 | /** 31 | * Same as ClickHouse's hash function `javaHash`.
32 | * ! Extended to integers from ClickHouse release 22.10. 33 | */ 34 | public class JavaHashPartitioner extends ClickHousePartitioner { 35 | 36 | private final FieldGetter fieldGetter; 37 | 38 | public JavaHashPartitioner(List getters) { 39 | checkArgument( 40 | getters.size() == 1 && nonNull(getters.get(0)), 41 | "The parameter number of JavaHashPartitioner must be 1"); 42 | this.fieldGetter = getters.get(0); 43 | } 44 | 45 | @Override 46 | public int select(RowData record, ClusterSpec clusterSpec) { 47 | long weightSum = clusterSpec.getWeightSum(); 48 | long result = Objects.hashCode(fieldGetter.getFieldOrNull(record)) % weightSum; 49 | return select(result, clusterSpec); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/partitioner/ShufflePartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.partitioner; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 21 | import org.apache.flink.connector.clickhouse.internal.schema.ShardSpec; 22 | import org.apache.flink.table.data.RowData; 23 | 24 | import java.util.List; 25 | import java.util.concurrent.ThreadLocalRandom; 26 | 27 | /** Shuffle data by random numbers. */ 28 | public class ShufflePartitioner extends ClickHousePartitioner { 29 | 30 | public ShufflePartitioner() {} 31 | 32 | @Override 33 | public int select(RowData record, ClusterSpec clusterSpec) { 34 | List shards = clusterSpec.getShards(); 35 | int index = ThreadLocalRandom.current().nextInt(shards.size()); 36 | return shards.get(index).getNum(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/partitioner/ValuePartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.partitioner; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.ClusterSpec; 21 | import org.apache.flink.table.data.DecimalData; 22 | import org.apache.flink.table.data.RowData; 23 | 24 | import java.math.BigInteger; 25 | import java.util.List; 26 | 27 | import static java.util.Objects.nonNull; 28 | import static org.apache.flink.util.Preconditions.checkArgument; 29 | 30 | /** Partition key value based, value must be a number. */ 31 | public class ValuePartitioner extends ClickHousePartitioner { 32 | 33 | private final RowData.FieldGetter fieldGetter; 34 | 35 | public ValuePartitioner(List getters) { 36 | checkArgument( 37 | getters.size() == 1 && nonNull(getters.get(0)), 38 | "The parameter number of ValuePartitioner must be 1"); 39 | this.fieldGetter = getters.get(0); 40 | } 41 | 42 | @Override 43 | public int select(RowData record, ClusterSpec clusterSpec) { 44 | Object value = fieldGetter.getFieldOrNull(record); 45 | long weightSum = clusterSpec.getWeightSum(); 46 | 47 | long num; 48 | if (value instanceof Byte) { 49 | num = (byte) value % weightSum; 50 | } else if (value instanceof Short) { 51 | num = (short) value % weightSum; 52 | } else if (value instanceof Integer) { 53 | num = (int) value % weightSum; 54 | } else if (value instanceof Long) { 55 | num = (int) ((long) value % weightSum); 56 | } else if (value instanceof Float) { 57 | num = (int) ((float) value % weightSum); 58 | } else if (value instanceof Double) { 59 | num = (int) ((double) value % weightSum); 60 | } else if (value instanceof DecimalData) { 61 | num = 62 | ((DecimalData) value) 63 | .toBigDecimal() 64 | .toBigInteger() 65 | .mod(BigInteger.valueOf(weightSum)) 66 | .intValue(); 67 | } else { 68 | Class valueClass = value == null ? null : value.getClass(); 69 | throw new RuntimeException("Unsupported number type: " + valueClass); 70 | } 71 | 72 | return select(num, clusterSpec); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/ClusterSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import javax.annotation.Nonnull; 21 | 22 | import java.io.Serializable; 23 | import java.util.ArrayList; 24 | import java.util.List; 25 | 26 | import static java.util.stream.Collectors.toList; 27 | import static org.apache.flink.util.Preconditions.checkNotNull; 28 | 29 | /** Cluster. */ 30 | public class ClusterSpec implements Serializable { 31 | private final String name; 32 | 33 | private final List shards; 34 | 35 | private final Long weightSum; 36 | 37 | public ClusterSpec(@Nonnull String name, @Nonnull List shards) { 38 | this.name = checkNotNull(name); 39 | 40 | List sortedList = new ArrayList<>(shards).stream().sorted().collect(toList()); 41 | this.shards = checkNotNull(sortedList); 42 | this.weightSum = sortedList.stream().mapToLong(ShardSpec::getWeight).sum(); 43 | 44 | sortedList.forEach( 45 | shardSpec -> { 46 | List weights = 47 | sortedList.stream().map(ShardSpec::getWeight).collect(toList()); 48 | shardSpec.initShardRangeBound(weights); 49 | }); 50 | } 51 | 52 | public String getName() { 53 | return name; 54 | } 55 | 56 | public List getShards() { 57 | return shards; 58 | } 59 | 60 | public Long getWeightSum() { 61 | return weightSum; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/DistributedEngineFull.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import java.io.Serializable; 21 | 22 | import static org.apache.flink.util.Preconditions.checkArgument; 23 | import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; 24 | 25 | /** Distributed table engine full schema. */ 26 | public class DistributedEngineFull implements Serializable { 27 | 28 | private final String cluster; 29 | 30 | private final String database; 31 | 32 | private final String table; 33 | 34 | private final Expression shardingKey; 35 | 36 | private final String policyName; 37 | 38 | public static DistributedEngineFull of( 39 | String cluster, 40 | String database, 41 | String table, 42 | Expression shardingKey, 43 | String policyName) { 44 | return new DistributedEngineFull(cluster, database, table, shardingKey, policyName); 45 | } 46 | 47 | private DistributedEngineFull( 48 | String cluster, 49 | String database, 50 | String table, 51 | Expression shardingKey, 52 | String policyName) { 53 | checkArgument(!isNullOrWhitespaceOnly(cluster), "cluster cannot be null or empty"); 54 | checkArgument(!isNullOrWhitespaceOnly(database), "database cannot be null or empty"); 55 | checkArgument(!isNullOrWhitespaceOnly(table), "table cannot be null or empty"); 56 | 57 | this.cluster = cluster; 58 | this.database = database; 59 | this.table = table; 60 | this.shardingKey = shardingKey; 61 | this.policyName = policyName; 62 | } 63 | 64 | public String getCluster() { 65 | return cluster; 66 | } 67 | 68 | public String getDatabase() { 69 | return database; 70 | } 71 | 72 | public String getTable() { 73 | return table; 74 | } 75 | 76 | public Expression getShardingKey() { 77 | return shardingKey; 78 | } 79 | 80 | public String getPolicyName() { 81 | return policyName; 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/Expression.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import java.io.Serializable; 21 | 22 | /** Expression. */ 23 | public abstract class Expression implements Serializable { 24 | 25 | private static final long serialVersionUID = 1L; 26 | 27 | public abstract String explain(); 28 | } 29 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/FieldExpr.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import javax.annotation.Nonnull; 21 | 22 | import static org.apache.flink.util.Preconditions.checkArgument; 23 | import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; 24 | 25 | /** Column expression. */ 26 | public class FieldExpr extends Expression { 27 | 28 | private final String columnName; 29 | 30 | private FieldExpr(String columnName) { 31 | checkArgument(!isNullOrWhitespaceOnly(columnName), "columnName cannot be null or empty"); 32 | this.columnName = columnName; 33 | } 34 | 35 | public static FieldExpr of(@Nonnull String columnName) { 36 | return new FieldExpr(columnName); 37 | } 38 | 39 | public String getColumnName() { 40 | return columnName; 41 | } 42 | 43 | @Override 44 | public String explain() { 45 | return columnName; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/FunctionExpr.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import javax.annotation.Nonnull; 21 | 22 | import java.util.List; 23 | 24 | import static java.util.stream.Collectors.joining; 25 | import static org.apache.flink.util.Preconditions.checkArgument; 26 | import static org.apache.flink.util.Preconditions.checkNotNull; 27 | import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; 28 | 29 | /** Function expression. */ 30 | public class FunctionExpr extends Expression { 31 | 32 | private final String functionName; 33 | 34 | private final List arguments; 35 | 36 | private FunctionExpr(String functionName, List arguments) { 37 | checkArgument( 38 | !isNullOrWhitespaceOnly(functionName), "functionName cannot be null or empty"); 39 | checkNotNull(arguments, "arguments cannot be null"); 40 | 41 | this.functionName = functionName; 42 | this.arguments = arguments; 43 | } 44 | 45 | public static FunctionExpr of( 46 | @Nonnull String functionName, @Nonnull List arguments) { 47 | return new FunctionExpr(functionName, arguments); 48 | } 49 | 50 | public String getFunctionName() { 51 | return functionName; 52 | } 53 | 54 | public List getArguments() { 55 | return arguments; 56 | } 57 | 58 | @Override 59 | public String explain() { 60 | String joinedArgs = arguments.stream().map(Expression::explain).collect(joining(",")); 61 | return String.format("%s(%s)", functionName, joinedArgs); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/ReplicaSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import javax.annotation.Nonnull; 21 | 22 | import java.io.Serializable; 23 | 24 | import static org.apache.flink.util.Preconditions.checkNotNull; 25 | 26 | /** Replica. */ 27 | public class ReplicaSpec implements Comparable, Serializable { 28 | 29 | private final Integer num; 30 | 31 | private final String host; 32 | 33 | private final Integer port; 34 | 35 | public ReplicaSpec(@Nonnull Integer num, @Nonnull String host, @Nonnull Integer port) { 36 | this.num = checkNotNull(num); 37 | this.host = checkNotNull(host); 38 | this.port = checkNotNull(port); 39 | } 40 | 41 | public Integer getNum() { 42 | return num; 43 | } 44 | 45 | public String getHost() { 46 | return host; 47 | } 48 | 49 | public Integer getPort() { 50 | return port; 51 | } 52 | 53 | @Override 54 | public int compareTo(ReplicaSpec replicaSpec) { 55 | return this.getNum() - replicaSpec.getNum(); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/internal/schema/ShardSpec.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.internal.schema; 19 | 20 | import org.apache.flink.util.Preconditions; 21 | 22 | import javax.annotation.Nonnull; 23 | 24 | import java.io.Serializable; 25 | import java.util.ArrayList; 26 | import java.util.List; 27 | 28 | import static java.util.stream.Collectors.joining; 29 | import static java.util.stream.Collectors.toList; 30 | import static org.apache.flink.util.Preconditions.checkNotNull; 31 | 32 | /** Shard. */ 33 | public class ShardSpec implements Comparable, Serializable { 34 | private final Integer num; 35 | 36 | private final Long weight; 37 | 38 | private final List replicas; 39 | 40 | private Long shardLowerBound; 41 | 42 | private Long shardUpperBound; 43 | 44 | public ShardSpec( 45 | @Nonnull Integer num, @Nonnull Long weight, @Nonnull List replicas) { 46 | this.num = checkNotNull(num); 47 | this.weight = checkNotNull(weight); 48 | this.replicas = checkNotNull(new ArrayList<>(replicas).stream().sorted().collect(toList())); 49 | } 50 | 51 | public String getJdbcUrls() { 52 | return replicas.stream() 53 | .map(replicaSpec -> replicaSpec.getHost() + ":" + replicaSpec.getPort()) 54 | .collect(joining(",", "jdbc:ch://", "")); 55 | } 56 | 57 | public void initShardRangeBound(List weights) { 58 | Preconditions.checkState( 59 | weights.size() >= this.num, 60 | "Shard number must be less than or equal to shard amount."); 61 | shardLowerBound = weights.stream().mapToLong(value -> value).limit(this.num - 1).sum(); 62 | shardUpperBound = weights.stream().mapToLong(value -> value).limit(this.num).sum(); 63 | } 64 | 65 | public boolean isInShardRangeBounds(long number) { 66 | return number >= shardLowerBound && number < shardUpperBound; 67 | } 68 | 69 | @Override 70 | public int compareTo(ShardSpec shardSpec) { 71 | return this.getNum() - shardSpec.getNum(); 72 | } 73 | 74 | public Integer getNum() { 75 | return num; 76 | } 77 | 78 | public Long getWeight() { 79 | return weight; 80 | } 81 | 82 | public List getReplicas() { 83 | return replicas; 84 | } 85 | 86 | public Long getShardLowerBound() { 87 | return shardLowerBound; 88 | } 89 | 90 | public Long getShardUpperBound() { 91 | return shardUpperBound; 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseBatchBetweenParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | import static org.apache.flink.util.Preconditions.checkArgument; 21 | 22 | /** For example, $columnName BETWEEN ? AND ? */ 23 | public class ClickHouseBatchBetweenParametersProvider extends ClickHouseBetweenParametersProvider { 24 | 25 | public ClickHouseBatchBetweenParametersProvider(long minVal, long maxVal) { 26 | super(minVal, maxVal); 27 | } 28 | 29 | @Override 30 | public ClickHouseBatchBetweenParametersProvider ofBatchNum(Integer batchNum) { 31 | checkArgument(batchNum != null && batchNum > 0, "Batch number must be positive"); 32 | 33 | long maxElemCount = defaultMaxIfLTZero((maxVal - minVal) + 1); 34 | if (batchNum > maxElemCount) { 35 | batchNum = (int) maxElemCount; 36 | } 37 | this.batchNum = batchNum; 38 | return this; 39 | } 40 | 41 | @Override 42 | public ClickHouseBatchBetweenParametersProvider calculate() { 43 | this.parameterValues = divideParameterValues(batchNum); 44 | return this; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseBetweenParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | import java.io.Serializable; 21 | 22 | import static org.apache.flink.util.Preconditions.checkArgument; 23 | import static org.apache.flink.util.Preconditions.checkState; 24 | 25 | /** This class is used to compute the list of parallel query to run (i.e. splits). */ 26 | public abstract class ClickHouseBetweenParametersProvider extends ClickHouseParametersProvider { 27 | 28 | private static final String BETWEEN_CLAUSE = "`%s` BETWEEN ? AND ?"; 29 | 30 | protected final long minVal; 31 | 32 | protected final long maxVal; 33 | 34 | public ClickHouseBetweenParametersProvider(long minVal, long maxVal) { 35 | checkArgument(maxVal >= minVal, "maxVal must be larger than minVal"); 36 | this.minVal = minVal; 37 | this.maxVal = maxVal; 38 | } 39 | 40 | @Override 41 | public String getParameterClause() { 42 | return BETWEEN_CLAUSE; 43 | } 44 | 45 | protected long defaultMaxIfLTZero(long value) { 46 | return value < 0 ? Long.MAX_VALUE : value; 47 | } 48 | 49 | protected Serializable[][] divideParameterValues(int batchNum) { 50 | long maxElemCount = defaultMaxIfLTZero((maxVal - minVal) + 1); 51 | long batchSize = new Double(Math.ceil((double) maxElemCount / batchNum)).longValue(); 52 | long bigBatchNum = maxElemCount - (batchSize - 1) * batchNum; 53 | 54 | checkState(batchSize > 0, "Batch size and batch number must be positive."); 55 | 56 | Serializable[][] parameters = new Serializable[batchNum][2]; 57 | long start = minVal; 58 | for (int i = 0; i < batchNum; i++) { 59 | long end = start + batchSize - 1 - (i >= bigBatchNum ? 1 : 0); 60 | end = defaultMaxIfLTZero(end); 61 | parameters[i] = new Long[] {start, end}; 62 | start = end + 1; 63 | } 64 | return parameters; 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseNonParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | /** Non parameters provided. */ 21 | public class ClickHouseNonParametersProvider extends ClickHouseParametersProvider { 22 | 23 | @Override 24 | public String getParameterClause() { 25 | return null; 26 | } 27 | 28 | @Override 29 | public ClickHouseParametersProvider ofBatchNum(Integer batchNum) { 30 | return this; 31 | } 32 | 33 | @Override 34 | public ClickHouseParametersProvider calculate() { 35 | return this; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | import java.io.Serializable; 21 | 22 | /** Clickhouse parameters provider. */ 23 | public abstract class ClickHouseParametersProvider { 24 | 25 | protected Serializable[][] parameterValues; 26 | protected Serializable[][] shardIdValues; 27 | protected int batchNum; 28 | 29 | /** Returns the necessary parameters array to use for query in parallel a table. */ 30 | public Serializable[][] getParameterValues() { 31 | return parameterValues; 32 | } 33 | 34 | /** Returns the shard ids that the parameter values act on. */ 35 | public Serializable[][] getShardIdValues() { 36 | return shardIdValues; 37 | } 38 | 39 | public abstract String getParameterClause(); 40 | 41 | public abstract ClickHouseParametersProvider ofBatchNum(Integer batchNum); 42 | 43 | public abstract ClickHouseParametersProvider calculate(); 44 | 45 | // -------------------------- Methods for local tables -------------------------- 46 | 47 | protected int[] allocateShards(int minBatchSize, int minBatchNum, int length) { 48 | int[] shards = new int[length]; 49 | for (int i = 0; i < length; i++) { 50 | if (i + 1 <= minBatchNum) { 51 | shards[i] = minBatchSize; 52 | } else { 53 | shards[i] = minBatchSize + 1; 54 | } 55 | } 56 | return shards; 57 | } 58 | 59 | protected Integer[] subShardIds(int start, int idNum, int[] shardIds) { 60 | Integer[] subIds = new Integer[idNum]; 61 | for (int i = 0; i < subIds.length; i++) { 62 | subIds[i] = shardIds[start + i]; 63 | } 64 | return subIds; 65 | } 66 | 67 | /** Builder. */ 68 | public static class Builder { 69 | 70 | private Long minVal; 71 | 72 | private Long maxVal; 73 | 74 | private Integer batchNum; 75 | 76 | private int[] shardIds; 77 | 78 | private boolean useLocal; 79 | 80 | public Builder setMinVal(Long minVal) { 81 | this.minVal = minVal; 82 | return this; 83 | } 84 | 85 | public Builder setMaxVal(Long maxVal) { 86 | this.maxVal = maxVal; 87 | return this; 88 | } 89 | 90 | public Builder setBatchNum(Integer batchNum) { 91 | this.batchNum = batchNum; 92 | return this; 93 | } 94 | 95 | public Builder setShardIds(int[] shardIds) { 96 | this.shardIds = shardIds; 97 | return this; 98 | } 99 | 100 | public Builder setUseLocal(boolean useLocal) { 101 | this.useLocal = useLocal; 102 | return this; 103 | } 104 | 105 | public ClickHouseParametersProvider build() { 106 | ClickHouseParametersProvider parametersProvider; 107 | if (minVal == null || maxVal == null) { 108 | parametersProvider = 109 | useLocal && shardIds != null 110 | ? new ClickHouseShardTableParametersProvider(shardIds) 111 | : new ClickHouseNonParametersProvider(); 112 | } else { 113 | parametersProvider = 114 | useLocal && shardIds != null 115 | ? new ClickHouseShardBetweenParametersProvider( 116 | minVal, maxVal, shardIds) 117 | : new ClickHouseBatchBetweenParametersProvider(minVal, maxVal); 118 | } 119 | 120 | return parametersProvider.ofBatchNum(batchNum).calculate(); 121 | } 122 | } 123 | } 124 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseShardBetweenParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | import org.apache.flink.annotation.Experimental; 21 | 22 | import org.apache.commons.lang3.ArrayUtils; 23 | 24 | import java.io.Serializable; 25 | import java.util.Arrays; 26 | 27 | import static org.apache.flink.util.Preconditions.checkArgument; 28 | 29 | /** For example, $columnName BETWEEN ? AND ? */ 30 | @Experimental 31 | public class ClickHouseShardBetweenParametersProvider extends ClickHouseBetweenParametersProvider { 32 | 33 | private final int[] shardIds; 34 | private final int shardNum; 35 | 36 | public ClickHouseShardBetweenParametersProvider(long minVal, long maxVal, int[] shardIds) { 37 | super(minVal, maxVal); 38 | 39 | checkArgument(shardIds.length > 1, "length of shardIds must be larger than 0"); 40 | this.shardIds = shardIds; 41 | this.shardNum = shardIds.length; 42 | } 43 | 44 | @Override 45 | public ClickHouseShardBetweenParametersProvider ofBatchNum(Integer batchNum) { 46 | checkArgument(batchNum != null && batchNum > 0, "batchNum must be positive"); 47 | 48 | long maxElemCount = Math.max(maxVal - minVal, 1) * shardNum + 1; 49 | maxElemCount = defaultMaxIfLTZero(maxElemCount); 50 | if (batchNum > maxElemCount) { 51 | batchNum = (int) maxElemCount; 52 | } 53 | this.batchNum = batchNum; 54 | return this; 55 | } 56 | 57 | @Override 58 | public ClickHouseShardBetweenParametersProvider calculate() { 59 | Serializable[][] parameters = null; 60 | Integer[][] shardIdValues = null; 61 | 62 | float factor = ((float) batchNum) / shardNum; 63 | if (factor >= 1) { 64 | // e.g. batchNum = 10, shardNum = 3. 65 | int minBatchSize = (int) factor; 66 | int minBatchNum = (minBatchSize + 1) * shardNum - batchNum; 67 | int[] info = allocateShards(minBatchSize, minBatchNum, shardNum); 68 | for (int i = 0; i < info.length; i++) { 69 | parameters = ArrayUtils.addAll(parameters, divideParameterValues(info[i])); 70 | shardIdValues = 71 | ArrayUtils.addAll(shardIdValues, repeatShardId(shardIds[i], info[i])); 72 | } 73 | } else if (factor < 1) { 74 | // e.g. batchNum = 10, shardNum = 23. 75 | int minBatchSize = (int) (1 / factor); 76 | int minBatchNum = (minBatchSize + 1) * batchNum - shardNum; 77 | int[] info = allocateShards(minBatchSize, minBatchNum, batchNum); 78 | for (int i = 0; i < info.length; i++) { 79 | int start = Arrays.stream(ArrayUtils.subarray(info, 0, i)).sum(); 80 | parameters = ArrayUtils.addAll(parameters, divideParameterValues(1)); 81 | shardIdValues = 82 | ArrayUtils.add(shardIdValues, subShardIds(start, info[i], shardIds)); 83 | } 84 | } 85 | 86 | this.parameterValues = parameters; 87 | this.shardIdValues = shardIdValues; 88 | return this; 89 | } 90 | 91 | private Integer[][] repeatShardId(int shardId, int shardNum) { 92 | Integer[][] shards = new Integer[shardNum][1]; 93 | for (int i = 0; i < shardNum; i++) { 94 | shards[i] = new Integer[] {shardId}; 95 | } 96 | return shards; 97 | } 98 | } 99 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/split/ClickHouseShardTableParametersProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.split; 19 | 20 | import org.apache.flink.annotation.Experimental; 21 | 22 | import org.apache.commons.lang3.ArrayUtils; 23 | 24 | import java.util.Arrays; 25 | 26 | import static org.apache.flink.util.Preconditions.checkArgument; 27 | 28 | /** For example, $columnName BETWEEN ? AND ? */ 29 | @Experimental 30 | public class ClickHouseShardTableParametersProvider extends ClickHouseParametersProvider { 31 | 32 | private final int[] shardIds; 33 | private final int shardNum; 34 | 35 | public ClickHouseShardTableParametersProvider(int[] shardIds) { 36 | checkArgument(shardIds.length > 0, "length of shardIds must be larger than 0"); 37 | this.shardIds = shardIds; 38 | this.shardNum = shardIds.length; 39 | } 40 | 41 | @Override 42 | public String getParameterClause() { 43 | return null; 44 | } 45 | 46 | @Override 47 | public ClickHouseShardTableParametersProvider ofBatchNum(Integer batchNum) { 48 | batchNum = batchNum != null ? batchNum : shardNum; 49 | checkArgument(batchNum > 0, "batchNum must be positive"); 50 | 51 | if (batchNum > shardNum) { 52 | batchNum = shardNum; 53 | } 54 | this.batchNum = batchNum; 55 | return this; 56 | } 57 | 58 | @Override 59 | public ClickHouseShardTableParametersProvider calculate() { 60 | int minBatchSize = shardNum / batchNum; 61 | int minBatchNum = (minBatchSize + 1) * batchNum - shardNum; 62 | int[] info = allocateShards(minBatchSize, minBatchNum, batchNum); 63 | 64 | Integer[][] shardIdValues = null; 65 | for (int i = 0; i < info.length; i++) { 66 | int start = Arrays.stream(ArrayUtils.subarray(info, 0, i)).sum(); 67 | shardIdValues = ArrayUtils.add(shardIdValues, subShardIds(start, info[i], shardIds)); 68 | } 69 | 70 | this.shardIdValues = shardIdValues; 71 | return this; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/util/ClickHouseUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.util; 19 | 20 | import org.apache.flink.connector.clickhouse.internal.schema.Expression; 21 | import org.apache.flink.connector.clickhouse.internal.schema.FieldExpr; 22 | import org.apache.flink.connector.clickhouse.internal.schema.FunctionExpr; 23 | 24 | import java.sql.Timestamp; 25 | import java.time.LocalDate; 26 | import java.time.LocalDateTime; 27 | import java.time.LocalTime; 28 | import java.util.ArrayList; 29 | import java.util.List; 30 | import java.util.Map; 31 | import java.util.Properties; 32 | import java.util.TimeZone; 33 | 34 | import static java.util.Collections.emptyList; 35 | import static java.util.Collections.singletonList; 36 | import static org.apache.flink.connector.clickhouse.config.ClickHouseConfig.PROPERTIES_PREFIX; 37 | import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; 38 | 39 | /** clickhouse util. */ 40 | public class ClickHouseUtil { 41 | 42 | public static final String EMPTY = ""; 43 | 44 | private static final LocalDate DATE_PREFIX_OF_TIME = LocalDate.ofEpochDay(1); 45 | 46 | public static Properties getClickHouseProperties(Map tableOptions) { 47 | final Properties properties = new Properties(); 48 | 49 | tableOptions.keySet().stream() 50 | .filter(key -> key.startsWith(PROPERTIES_PREFIX)) 51 | .forEach( 52 | key -> { 53 | final String value = tableOptions.get(key); 54 | final String subKey = key.substring((PROPERTIES_PREFIX).length()); 55 | properties.setProperty(subKey, value); 56 | }); 57 | return properties; 58 | } 59 | 60 | public static Timestamp toEpochDayOneTimestamp(LocalTime localTime) { 61 | LocalDateTime localDateTime = localTime.atDate(DATE_PREFIX_OF_TIME); 62 | return Timestamp.valueOf(localDateTime); 63 | } 64 | 65 | public static LocalDateTime toLocalDateTime(LocalTime localTime) { 66 | return localTime.atDate(DATE_PREFIX_OF_TIME); 67 | } 68 | 69 | public static String quoteIdentifier(String identifier) { 70 | return String.join(EMPTY, "`", identifier, "`"); 71 | } 72 | 73 | public static Expression parseShardingKey(String shardingKey) { 74 | if (isNullOrWhitespaceOnly(shardingKey)) { 75 | return null; 76 | } 77 | 78 | if (!shardingKey.contains("(")) { 79 | return FieldExpr.of(shardingKey); 80 | } 81 | 82 | return parseFunctionExpr(shardingKey); 83 | } 84 | 85 | private static Expression parseFunctionExpr(String shardingExpr) { 86 | int bracketStartIndex = shardingExpr.indexOf("("); 87 | String functionName = shardingExpr.substring(0, bracketStartIndex); 88 | String subExprLiteral = 89 | shardingExpr.substring(bracketStartIndex + 1, shardingExpr.lastIndexOf(")")); 90 | 91 | if (subExprLiteral.trim().isEmpty()) { 92 | return FunctionExpr.of(functionName, emptyList()); 93 | } 94 | 95 | if (!subExprLiteral.contains("(")) { 96 | String[] subExprLiteralList = subExprLiteral.split(","); 97 | List exprList = new ArrayList<>(subExprLiteralList.length); 98 | for (String exprLiteral : subExprLiteralList) { 99 | exprList.add(FieldExpr.of(exprLiteral)); 100 | } 101 | return FunctionExpr.of(functionName, exprList); 102 | } 103 | 104 | Expression expression = parseFunctionExpr(subExprLiteral); 105 | return FunctionExpr.of(functionName, singletonList(expression)); 106 | } 107 | 108 | /** TODO The timezone configured via `table.local-time-zone` should be used. */ 109 | public static TimeZone getFlinkTimeZone() { 110 | return TimeZone.getDefault(); 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/util/DataTypeUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.util; 19 | 20 | import org.apache.flink.api.java.tuple.Tuple2; 21 | import org.apache.flink.table.api.DataTypes; 22 | import org.apache.flink.table.catalog.exceptions.CatalogException; 23 | import org.apache.flink.table.types.DataType; 24 | 25 | import com.clickhouse.data.ClickHouseColumn; 26 | 27 | import java.util.regex.Matcher; 28 | import java.util.regex.Pattern; 29 | import java.util.stream.Collectors; 30 | 31 | import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION; 32 | 33 | /** Type utils. */ 34 | public class DataTypeUtil { 35 | 36 | private static final Pattern INTERNAL_TYPE_PATTERN = Pattern.compile(".*?\\((?.*)\\)"); 37 | 38 | /** 39 | * Convert clickhouse data type to flink data type.
40 | * TODO: Whether to indicate nullable? 41 | */ 42 | public static DataType toFlinkType(ClickHouseColumn clickHouseColumnInfo) { 43 | switch (clickHouseColumnInfo.getDataType()) { 44 | case Int8: 45 | return DataTypes.TINYINT(); 46 | case Bool: 47 | return DataTypes.BOOLEAN(); 48 | case Int16: 49 | case UInt8: 50 | return DataTypes.SMALLINT(); 51 | case Int32: 52 | case UInt16: 53 | case IntervalYear: 54 | case IntervalMonth: 55 | case IntervalWeek: 56 | case IntervalDay: 57 | case IntervalHour: 58 | case IntervalQuarter: 59 | case IntervalMinute: 60 | case IntervalSecond: 61 | return DataTypes.INT(); 62 | case Int64: 63 | case UInt32: 64 | return DataTypes.BIGINT(); 65 | case Int128: 66 | case Int256: 67 | case UInt64: 68 | case UInt128: 69 | case UInt256: 70 | return DataTypes.DECIMAL(MAX_PRECISION, 0); 71 | case Float32: 72 | return DataTypes.FLOAT(); 73 | case Float64: 74 | return DataTypes.DOUBLE(); 75 | case Decimal: 76 | return DataTypes.DECIMAL( 77 | clickHouseColumnInfo.getPrecision(), clickHouseColumnInfo.getScale()); 78 | case Decimal32: 79 | return DataTypes.DECIMAL(9, clickHouseColumnInfo.getScale()); 80 | case Decimal64: 81 | return DataTypes.DECIMAL(18, clickHouseColumnInfo.getScale()); 82 | case Decimal128: 83 | case Decimal256: 84 | return DataTypes.DECIMAL( 85 | Math.min(MAX_PRECISION, clickHouseColumnInfo.getPrecision()), 86 | Math.min(MAX_PRECISION, clickHouseColumnInfo.getScale())); 87 | case String: 88 | case Enum8: 89 | case Enum16: 90 | return DataTypes.STRING(); 91 | case FixedString: 92 | case IPv4: 93 | case IPv6: 94 | case UUID: 95 | return DataTypes.VARCHAR(clickHouseColumnInfo.getPrecision()); 96 | case Date: 97 | case Date32: 98 | return DataTypes.DATE(); 99 | case DateTime: 100 | case DateTime32: 101 | case DateTime64: 102 | return DataTypes.TIMESTAMP(clickHouseColumnInfo.getScale()); 103 | case Array: 104 | String arrayBaseType = 105 | getInternalClickHouseType(clickHouseColumnInfo.getOriginalTypeName()); 106 | String arrayBaseName = clickHouseColumnInfo.getColumnName() + ".array_base"; 107 | ClickHouseColumn clickHouseColumn = 108 | ClickHouseColumn.of(arrayBaseName, arrayBaseType); 109 | return DataTypes.ARRAY(toFlinkType(clickHouseColumn)); 110 | case Map: 111 | return DataTypes.MAP( 112 | toFlinkType(clickHouseColumnInfo.getKeyInfo()), 113 | toFlinkType(clickHouseColumnInfo.getValueInfo())); 114 | case Tuple: 115 | return DataTypes.ROW( 116 | clickHouseColumnInfo.getNestedColumns().stream() 117 | .map((col) -> new Tuple2<>(col, toFlinkType(col))) 118 | .map(tuple -> DataTypes.FIELD(tuple.f0.getColumnName(), tuple.f1)) 119 | .collect(Collectors.toList())); 120 | 121 | case Nested: 122 | case AggregateFunction: 123 | default: 124 | throw new UnsupportedOperationException( 125 | "Unsupported type:" + clickHouseColumnInfo.getDataType()); 126 | } 127 | } 128 | 129 | private static String getInternalClickHouseType(String clickHouseTypeLiteral) { 130 | Matcher matcher = INTERNAL_TYPE_PATTERN.matcher(clickHouseTypeLiteral); 131 | if (matcher.find()) { 132 | return matcher.group("type"); 133 | } else { 134 | throw new CatalogException( 135 | String.format("No content found in the bucket of '%s'", clickHouseTypeLiteral)); 136 | } 137 | } 138 | } 139 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/util/SqlClause.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.clickhouse.util; 19 | 20 | import java.util.function.Function; 21 | 22 | /** SQL filters that support push down. */ 23 | public enum SqlClause { 24 | EQ(args -> String.format("%s = %s", args[0], args[1])), 25 | 26 | NOT_EQ(args -> String.format("%s <> %s", args[0], args[1])), 27 | 28 | GT(args -> String.format("%s > %s", args[0], args[1])), 29 | 30 | GT_EQ(args -> String.format("%s >= %s", args[0], args[1])), 31 | 32 | LT(args -> String.format("%s < %s", args[0], args[1])), 33 | 34 | LT_EQ(args -> String.format("%s <= %s", args[0], args[1])), 35 | 36 | IS_NULL(args -> String.format("%s IS NULL", args[0])), 37 | 38 | IS_NOT_NULL(args -> String.format("%s IS NOT NULL", args[0])), 39 | 40 | AND(args -> String.format("%s AND %s", args[0], args[1])), 41 | 42 | OR(args -> String.format("%s OR %s", args[0], args[1])); 43 | 44 | public final Function formatter; 45 | 46 | SqlClause(final Function function) { 47 | this.formatter = function; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /flink-connector-clickhouse/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.clickhouse.ClickHouseDynamicTableFactory 17 | org.apache.flink.connector.clickhouse.catalog.ClickHouseCatalogFactory 18 | -------------------------------------------------------------------------------- /flink-sql-connector-clickhouse/pom.xml: -------------------------------------------------------------------------------- 1 | 17 | 20 | 4.0.0 21 | 22 | org.apache.flink 23 | flink-connector-clickhouse-parent 24 | 1.0.0-SNAPSHOT 25 | 26 | 27 | flink-sql-connector-clickhouse 28 | Flink : Connectors : SQL : Clickhouse 29 | https://flink.apache.org 30 | jar 31 | 32 | 33 | UTF-8 34 | 35 | 36 | 37 | 38 | org.apache.flink 39 | flink-connector-clickhouse 40 | ${project.version} 41 | 42 | 43 | 44 | 45 | 46 | org.apache.maven.plugins 47 | maven-shade-plugin 48 | 49 | 50 | package 51 | 52 | shade 53 | 54 | 55 | false 56 | 57 | 58 | 59 | log4j:* 60 | org.slf4j:* 61 | com.google.code.findbugs:jsr305 62 | 63 | 64 | 65 | 66 | com.clickhouse 67 | ${shade.base}.com.clickhouse 68 | 69 | 70 | org.apache.commons 71 | ${shade.base}.org.apache.commons 72 | 73 | 74 | org.apache.hc.client5 75 | ${shade.base}.org.apache.hc.client5 76 | 77 | 78 | org.apache.hc.core5 79 | ${shade.base}.org.apache.hc.core5 80 | 81 | 82 | 83 | 84 | *:* 85 | 86 | META-INF/*.SF 87 | META-INF/*.DSA 88 | META-INF/*.RSA 89 | 90 | 91 | 92 | 93 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | org.apache.maven.plugins 102 | maven-release-plugin 103 | ${maven-release-plugin.version} 104 | 105 | release-@{project.version} 106 | true 107 | 108 | 109 | 110 | 111 | 112 | -------------------------------------------------------------------------------- /mvnw.cmd: -------------------------------------------------------------------------------- 1 | @REM ---------------------------------------------------------------------------- 2 | @REM Licensed to the Apache Software Foundation (ASF) under one 3 | @REM or more contributor license agreements. See the NOTICE file 4 | @REM distributed with this work for additional information 5 | @REM regarding copyright ownership. The ASF licenses this file 6 | @REM to you under the Apache License, Version 2.0 (the 7 | @REM "License"); you may not use this file except in compliance 8 | @REM with the License. You may obtain a copy of the License at 9 | @REM 10 | @REM http://www.apache.org/licenses/LICENSE-2.0 11 | @REM 12 | @REM Unless required by applicable law or agreed to in writing, 13 | @REM software distributed under the License is distributed on an 14 | @REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | @REM KIND, either express or implied. See the License for the 16 | @REM specific language governing permissions and limitations 17 | @REM under the License. 18 | @REM ---------------------------------------------------------------------------- 19 | 20 | @REM ---------------------------------------------------------------------------- 21 | @REM Maven Start Up Batch script 22 | @REM 23 | @REM Required ENV vars: 24 | @REM JAVA_HOME - location of a JDK home dir 25 | @REM 26 | @REM Optional ENV vars 27 | @REM M2_HOME - location of maven2's installed home dir 28 | @REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands 29 | @REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending 30 | @REM MAVEN_OPTS - parameters passed to the Java VM when running Maven 31 | @REM e.g. to debug Maven itself, use 32 | @REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 33 | @REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files 34 | @REM ---------------------------------------------------------------------------- 35 | 36 | @REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on' 37 | @echo off 38 | @REM set title of command window 39 | title %0 40 | @REM enable echoing by setting MAVEN_BATCH_ECHO to 'on' 41 | @if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO% 42 | 43 | @REM set %HOME% to equivalent of $HOME 44 | if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%") 45 | 46 | @REM Execute a user defined script before this one 47 | if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre 48 | @REM check for pre script, once with legacy .bat ending and once with .cmd ending 49 | if exist "%USERPROFILE%\mavenrc_pre.bat" call "%USERPROFILE%\mavenrc_pre.bat" %* 50 | if exist "%USERPROFILE%\mavenrc_pre.cmd" call "%USERPROFILE%\mavenrc_pre.cmd" %* 51 | :skipRcPre 52 | 53 | @setlocal 54 | 55 | set ERROR_CODE=0 56 | 57 | @REM To isolate internal variables from possible post scripts, we use another setlocal 58 | @setlocal 59 | 60 | @REM ==== START VALIDATION ==== 61 | if not "%JAVA_HOME%" == "" goto OkJHome 62 | 63 | echo. 64 | echo Error: JAVA_HOME not found in your environment. >&2 65 | echo Please set the JAVA_HOME variable in your environment to match the >&2 66 | echo location of your Java installation. >&2 67 | echo. 68 | goto error 69 | 70 | :OkJHome 71 | if exist "%JAVA_HOME%\bin\java.exe" goto init 72 | 73 | echo. 74 | echo Error: JAVA_HOME is set to an invalid directory. >&2 75 | echo JAVA_HOME = "%JAVA_HOME%" >&2 76 | echo Please set the JAVA_HOME variable in your environment to match the >&2 77 | echo location of your Java installation. >&2 78 | echo. 79 | goto error 80 | 81 | @REM ==== END VALIDATION ==== 82 | 83 | :init 84 | 85 | @REM Find the project base dir, i.e. the directory that contains the folder ".mvn". 86 | @REM Fallback to current working directory if not found. 87 | 88 | set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR% 89 | IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir 90 | 91 | set EXEC_DIR=%CD% 92 | set WDIR=%EXEC_DIR% 93 | :findBaseDir 94 | IF EXIST "%WDIR%"\.mvn goto baseDirFound 95 | cd .. 96 | IF "%WDIR%"=="%CD%" goto baseDirNotFound 97 | set WDIR=%CD% 98 | goto findBaseDir 99 | 100 | :baseDirFound 101 | set MAVEN_PROJECTBASEDIR=%WDIR% 102 | cd "%EXEC_DIR%" 103 | goto endDetectBaseDir 104 | 105 | :baseDirNotFound 106 | set MAVEN_PROJECTBASEDIR=%EXEC_DIR% 107 | cd "%EXEC_DIR%" 108 | 109 | :endDetectBaseDir 110 | 111 | IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig 112 | 113 | @setlocal EnableExtensions EnableDelayedExpansion 114 | for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a 115 | @endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS% 116 | 117 | :endReadAdditionalConfig 118 | 119 | SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe" 120 | set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar" 121 | set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain 122 | 123 | set DOWNLOAD_URL="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" 124 | 125 | FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO ( 126 | IF "%%A"=="wrapperUrl" SET DOWNLOAD_URL=%%B 127 | ) 128 | 129 | @REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central 130 | @REM This allows using the maven wrapper in projects that prohibit checking in binary data. 131 | if exist %WRAPPER_JAR% ( 132 | if "%MVNW_VERBOSE%" == "true" ( 133 | echo Found %WRAPPER_JAR% 134 | ) 135 | ) else ( 136 | if not "%MVNW_REPOURL%" == "" ( 137 | SET DOWNLOAD_URL="%MVNW_REPOURL%/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" 138 | ) 139 | if "%MVNW_VERBOSE%" == "true" ( 140 | echo Couldn't find %WRAPPER_JAR%, downloading it ... 141 | echo Downloading from: %DOWNLOAD_URL% 142 | ) 143 | 144 | powershell -Command "&{"^ 145 | "$webclient = new-object System.Net.WebClient;"^ 146 | "if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^ 147 | "$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^ 148 | "}"^ 149 | "[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%DOWNLOAD_URL%', '%WRAPPER_JAR%')"^ 150 | "}" 151 | if "%MVNW_VERBOSE%" == "true" ( 152 | echo Finished downloading %WRAPPER_JAR% 153 | ) 154 | ) 155 | @REM End of extension 156 | 157 | @REM Provide a "standardized" way to retrieve the CLI args that will 158 | @REM work with both Windows and non-Windows executions. 159 | set MAVEN_CMD_LINE_ARGS=%* 160 | 161 | %MAVEN_JAVA_EXE% ^ 162 | %JVM_CONFIG_MAVEN_PROPS% ^ 163 | %MAVEN_OPTS% ^ 164 | %MAVEN_DEBUG_OPTS% ^ 165 | -classpath %WRAPPER_JAR% ^ 166 | "-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" ^ 167 | %WRAPPER_LAUNCHER% %MAVEN_CONFIG% %* 168 | if ERRORLEVEL 1 goto error 169 | goto end 170 | 171 | :error 172 | set ERROR_CODE=1 173 | 174 | :end 175 | @endlocal & set ERROR_CODE=%ERROR_CODE% 176 | 177 | if not "%MAVEN_SKIP_RC%"=="" goto skipRcPost 178 | @REM check for post script, once with legacy .bat ending and once with .cmd ending 179 | if exist "%USERPROFILE%\mavenrc_post.bat" call "%USERPROFILE%\mavenrc_post.bat" 180 | if exist "%USERPROFILE%\mavenrc_post.cmd" call "%USERPROFILE%\mavenrc_post.cmd" 181 | :skipRcPost 182 | 183 | @REM pause the script if MAVEN_BATCH_PAUSE is set to 'on' 184 | if "%MAVEN_BATCH_PAUSE%"=="on" pause 185 | 186 | if "%MAVEN_TERMINATE_CMD%"=="on" exit %ERROR_CODE% 187 | 188 | cmd /C exit /B %ERROR_CODE% 189 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 21 | 4.0.0 22 | 23 | 24 | org.apache.flink 25 | flink-connector-parent 26 | 1.1.0 27 | 28 | 29 | flink-connector-clickhouse-parent 30 | 1.0.0-SNAPSHOT 31 | 32 | Flink : Connectors : Clickhouse Parent 33 | 34 | flink-connector-clickhouse 35 | flink-sql-connector-clickhouse 36 | flink-connector-clickhouse-e2e-test 37 | 38 | pom 39 | https://flink.apache.org 40 | 2024 41 | 42 | 43 | 44 | The Apache Software License, Version 2.0 45 | https://www.apache.org/licenses/LICENSE-2.0.txt 46 | repo 47 | 48 | 49 | 50 | 51 | https://github.com/itinycheng/flink-connector-clickhouse 52 | scm:git:git@github.com:itinycheng/flink-connector-clickhouse.git 53 | scm:git:git@github.com:itinycheng/flink-connector-clickhouse.git 54 | 55 | HEAD 56 | 57 | 58 | 59 | UTF-8 60 | 4.13.2 61 | 0.6.4 62 | 2.12 63 | 1.19.0 64 | 3.13.0 65 | 1.19.8 66 | 5.2.1 67 | 5.2 68 | org.apache.flink.shaded.clickhouse 69 | flink-connector-clickhouse-parent 70 | false 71 | 2.5.3 72 | 73 | 74 | 75 | 76 | junit 77 | junit 78 | ${junit.version} 79 | test 80 | 81 | 82 | 83 | 84 | 85 | 86 | com.clickhouse 87 | clickhouse-jdbc 88 | ${clickhouse-jdbc.version} 89 | 90 | 91 | org.apache.httpcomponents.client5 92 | httpclient5 93 | ${httpclient5.version} 94 | 95 | 96 | org.apache.commons 97 | commons-lang3 98 | ${commons-lang3.version} 99 | 100 | 101 | 102 | org.apache.flink 103 | flink-connector-base 104 | ${flink.version} 105 | provided 106 | 107 | 108 | org.apache.flink 109 | flink-table-api-java-bridge 110 | ${flink.version} 111 | provided 112 | true 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | org.codehaus.mojo 121 | exec-maven-plugin 122 | false 123 | 124 | 125 | org.apache.flink 126 | flink-ci-tools 127 | ${flink.version} 128 | 129 | 130 | 131 | 132 | 133 | org.apache.maven.plugins 134 | maven-jar-plugin 135 | 136 | 137 | 138 | 139 | com.github.siom79.japicmp 140 | japicmp-maven-plugin 141 | 0.20.0 142 | 143 | 144 | 145 | org.apache.rat 146 | apache-rat-plugin 147 | false 148 | 149 | 150 | org.apache.maven.plugins 151 | maven-checkstyle-plugin 152 | 153 | 154 | com.diffplug.spotless 155 | spotless-maven-plugin 156 | 157 | ${spotless.skip} 158 | 159 | 160 | 161 | org.apache.maven.plugins 162 | maven-compiler-plugin 163 | 164 | 165 | org.apache.maven.plugins 166 | maven-surefire-plugin 167 | 168 | 169 | org.apache.maven.plugins 170 | maven-enforcer-plugin 171 | 172 | 173 | org.apache.maven.plugins 174 | maven-shade-plugin 175 | 176 | 177 | org.commonjava.maven.plugins 178 | directory-maven-plugin 179 | 180 | 181 | 182 | 183 | 184 | 185 | java11 186 | 187 | [11,) 188 | 189 | 190 | false 191 | 11 192 | 193 | 194 | 195 | java17 196 | 197 | [17,) 198 | 199 | 200 | false 201 | 17 202 | 203 | 204 | 205 | java21 206 | 207 | [21,) 208 | 209 | 210 | true 211 | 21 212 | 213 | 214 | 215 | 216 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | 28 | -------------------------------------------------------------------------------- /tools/releasing/shared/.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | *.iml -------------------------------------------------------------------------------- /tools/releasing/shared/README.md: -------------------------------------------------------------------------------- 1 | This is a collection of release utils for [Apache Flink](https://flink.apache.org/) connectors. 2 | 3 | # Integration 4 | 5 | The scripts assume that they are integrated into a connector repo as a submodule into the connector repo 6 | under `tools/releasing/shared`. 7 | 8 | # Usage 9 | 10 | Some scripts rely on environment variables to be set. 11 | These are checked at the start of each script. 12 | Any instance of `${some_variable}` in this document refers to an environment variable that is used by the respective 13 | script. 14 | 15 | ## check_environment.sh 16 | 17 | Runs some pre-release checks for the current environment, for example that all required programs are available. 18 | This should be run once at the start of the release process. 19 | 20 | ## release_snapshot_branch.sh 21 | 22 | Creates (and pushes!) a new snapshot branch for the current commit. 23 | The branch name is automatically determined from the version in the pom. 24 | This script should be called when work on a new major/minor version of the connector has started. 25 | 26 | ## update_branch_version.sh 27 | 28 | Updates the version in the poms of the current branch to `${NEW_VERSION}`. 29 | 30 | ## stage_source_release.sh 31 | 32 | Creates a source release from the current branch and pushes it via `svn` 33 | to [dist.apache.org](https://dist.apache.org/repos/dist/dev/flink). 34 | The version is automatically determined from the version in the pom. 35 | The created `svn` directory will contain a `-rc${RC_NUM}` suffix. 36 | 37 | ## stage_jars.sh 38 | 39 | Creates the jars from the current branch and deploys them to [repository.apache.org](https://repository.apache.org). 40 | The version will be suffixed with the Flink minor version, extracted from`${FLINK_VERSION}`, to indicate the supported Flink version. 41 | If a particular version of a connector supports multiple Flink versions then this script should be called multiple 42 | times. 43 | 44 | ## release_source_release.sh 45 | 46 | Copies the source release from the [SVN release directory](https://dist.apache.org/repops/dist/dev/flink) to the 47 | [SVN release directory](https://dist.apache.org/repops/dist/release/flink) on [dist.apache.org](https://dist.apache.org). 48 | 49 | For safety purposes this script does not automatically determine the project and version from the current directory/branch/tag. 50 | 51 | ``` 52 | PROJECT=flink-connector-elasticsearch VERSION=3.0.0 RC_NUM=2 ./release_source_release.sh 53 | ``` 54 | 55 | ## release_git_tag.sh 56 | 57 | Creates a release tag for the current branch and pushes it to GitHub. 58 | The tag will be suffixed with `-rc${RC_NUM}`, if `${RC_NUM}` was set. 59 | This script should only be used _after_ the `-SNAPSHOT` version suffix was removd via `update_branch_version.sh`. 60 | 61 | ## update_japicmp_configuration.sh 62 | 63 | Sets the japicmp reference version in the pom of the current branch to `${NEW_VERSION}`, enables compatibility checks 64 | for `@PublicEvolving` when used on snapshot branches an clears the list of exclusions. 65 | This should be called after a release on the associated snapshot branch. If it was a minor release it should 66 | additionally be called on the `main` branch. 67 | 68 | # Common workflow 69 | 70 | 1. run `release_snapshot_branch.sh` 71 | 2. do some development work on the created snapshot branch 72 | 3. checkout a specific commit to create a release from 73 | 4. run `check_environment.sh` 74 | 5. run `update_branch_version.sh` 75 | 6. run `stage_source_release.sh` 76 | 7. run `stage_jars.sh` (once for each supported Flink version) 77 | 8. run `release_git_tag.sh` (with `RC_NUM`) 78 | 9. vote on release 79 | 10. finalize release or cancel and go back to step 2 80 | 11. run `release_source_release.sh` 81 | 12. run `release_git_tag.sh` (without `RC_NUM`) 82 | 13. run `update_japicmp_configuration.sh` (on snapshot branch, and maybe `main`) 83 | 84 | # Script naming conventions 85 | 86 | | Prefix | Meaning | 87 | |---------|------------------------------------------------------------------------| 88 | | check | Verifies conditions without making any changes. | 89 | | update | Applies modifications locally to the current branch. | 90 | | stage | Publishes an artifact to an intermediate location for voting purposes. | 91 | | release | Publishes an artifact to a user-facing location. | 92 | -------------------------------------------------------------------------------- /tools/releasing/shared/_init.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | # all scripts should contain this line + source ${SCRIPT_DIR}/_init.sh 21 | SCRIPT_DIR=$( cd -- "$( dirname -- "${BASH_SOURCE[0]}" )" &> /dev/null && pwd ) 22 | 23 | set -o errexit 24 | set -o nounset 25 | set -o pipefail 26 | 27 | export SHELLOPTS 28 | 29 | ########################### 30 | 31 | MVN=${MVN:-mvn} 32 | 33 | if [ "$(uname)" == "Darwin" ]; then 34 | SHASUM="shasum -a 512" 35 | else 36 | SHASUM="sha512sum" 37 | fi 38 | 39 | REMOTE=${REMOTE:-upstream} 40 | 41 | ########################## 42 | 43 | SOURCE_DIR=$( cd -- "$( dirname -- "${SCRIPT_DIR}/../../../.." )" &> /dev/null && pwd ) 44 | RELEASE_DIR=${SOURCE_DIR}/tools/releasing/release 45 | ARTIFACTS_DIR=${SOURCE_DIR}/tools/releasing/release/artifacts 46 | 47 | SVN_DEV_DIR="https://dist.apache.org/repos/dist/dev/flink" 48 | SVN_RELEASE_DIR="https://dist.apache.org/repos/dist/release/flink" -------------------------------------------------------------------------------- /tools/releasing/shared/_utils.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | function check_variables_set { 21 | any_missing=false 22 | 23 | for variable in "$@" 24 | do 25 | if [ -z "${!variable:-}" ]; then 26 | echo "${variable} was not set." 27 | any_missing=true 28 | fi 29 | done 30 | 31 | if [ ${any_missing} == true ]; then 32 | exit 1 33 | fi 34 | } 35 | 36 | function create_pristine_source { 37 | source_dir=$1 38 | release_dir=$2 39 | 40 | clone_dir="${release_dir}/tmp-clone" 41 | clean_dir="${release_dir}/tmp-clean-clone" 42 | rm -rf ${clone_dir} 43 | rm -rf ${clean_dir} 44 | # create a temporary git clone to ensure that we have a pristine source release 45 | git clone "${source_dir}" "${clone_dir}" 46 | 47 | rsync -a \ 48 | --exclude ".git" --exclude ".gitignore" --exclude ".gitattributes" --exclude ".gitmodules" --exclude ".github" \ 49 | --exclude ".idea" --exclude "*.iml" \ 50 | --exclude ".DS_Store" \ 51 | --exclude ".asf.yaml" \ 52 | --exclude "target" --exclude "tools/releasing/shared" \ 53 | "${clone_dir}/" "${clean_dir}" 54 | 55 | rm -rf "${clone_dir}" 56 | 57 | echo "${clean_dir}" 58 | } 59 | 60 | function get_pom_version { 61 | echo $(${MVN} help:evaluate -Dexpression="project.version" -q -DforceStdout) 62 | } 63 | 64 | function set_pom_version { 65 | new_version=$1 66 | 67 | ${MVN} org.codehaus.mojo:versions-maven-plugin:2.8.1:set -DnewVersion=${new_version} -DgenerateBackupPoms=false --quiet 68 | } -------------------------------------------------------------------------------- /tools/releasing/shared/check_environment.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | 24 | EXIT_CODE=0 25 | 26 | function check_program_available { 27 | if program=$(command -v ${1}); then 28 | printf "\t%-10s%s\n" "${1}" "using ${program}" 29 | else 30 | printf "\t%-10s%s\n" "${1}" "is not available." 31 | EXIT_CODE=1 32 | fi 33 | } 34 | 35 | echo "Checking program availability:" 36 | check_program_available git 37 | check_program_available tar 38 | check_program_available rsync 39 | check_program_available gpg 40 | check_program_available perl 41 | check_program_available sed 42 | check_program_available svn 43 | check_program_available ${MVN} 44 | check_program_available ${SHASUM} 45 | 46 | function check_git_connectivity { 47 | cd "${SOURCE_DIR}" 48 | if git ls-remote --exit-code ${REMOTE} &> /dev/null; then 49 | printf "\tUsing git remote '${REMOTE}'.\n" 50 | else 51 | printf "\tGit remote '${REMOTE}' is not available.\n" 52 | printf "\tRun 'git remote add upstream https://github.com/apache/' or set a custom remote with the 'REMOTE' env variable.\n" 53 | exit 1 54 | fi 55 | } 56 | 57 | echo "Checking git remote availability:" 58 | if ! (check_git_connectivity); then 59 | EXIT_CODE=1 60 | fi 61 | 62 | if [ ${EXIT_CODE} == 0 ]; then 63 | echo "All set! :)" 64 | else 65 | echo "At least one problem was found!" 66 | fi 67 | exit ${EXIT_CODE} 68 | -------------------------------------------------------------------------------- /tools/releasing/shared/release_git_tag.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source ${SCRIPT_DIR}/_init.sh 23 | source ${SCRIPT_DIR}/_utils.sh 24 | 25 | ########################### 26 | 27 | RC_NUM=${RC_NUM:-none} 28 | 29 | ########################### 30 | 31 | function create_release_tag { 32 | cd "${SOURCE_DIR}" 33 | 34 | version=$(get_pom_version) 35 | if [[ ${version} =~ -SNAPSHOT$ ]]; then 36 | echo "Tags should not be created for SNAPSHOT versions. Use 'update_branch_version.sh' first." 37 | exit 1 38 | fi 39 | 40 | tag=v${version} 41 | if [ "$RC_NUM" != "none" ]; then 42 | tag=${tag}-rc${RC_NUM} 43 | fi 44 | 45 | git tag -s -m "v${tag}" ${tag} 46 | 47 | git push ${REMOTE} ${tag} 48 | } 49 | 50 | (create_release_tag) 51 | -------------------------------------------------------------------------------- /tools/releasing/shared/release_snapshot_branch.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | 24 | ########################### 25 | 26 | function create_snapshot_branch { 27 | cd "${SOURCE_DIR}" 28 | 29 | version=$(mvn help:evaluate -Dexpression="project.version" -q -DforceStdout | sed "s/-SNAPSHOT//") 30 | branch="v${version}" 31 | 32 | git checkout -b ${branch} 33 | git push ${REMOTE} ${branch}:${branch} 34 | } 35 | 36 | (create_snapshot_branch) 37 | -------------------------------------------------------------------------------- /tools/releasing/shared/release_source_release.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | source "${SCRIPT_DIR}/_utils.sh" 24 | 25 | ########################### 26 | 27 | check_variables_set PROJECT VERSION RC_NUM 28 | 29 | ########################### 30 | 31 | function release_source_release { 32 | svn move -m "Release ${PROJECT} ${VERSION}" ${SVN_DEV_DIR}/${PROJECT}-${VERSION}-rc${RC_NUM} ${SVN_RELEASE_DIR}/${PROJECT}-${VERSION} 33 | } 34 | 35 | (release_source_release) 36 | -------------------------------------------------------------------------------- /tools/releasing/shared/stage_jars.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | source "${SCRIPT_DIR}/_utils.sh" 24 | 25 | ########################### 26 | 27 | check_variables_set FLINK_VERSION 28 | 29 | ########################### 30 | 31 | function deploy_staging_jars { 32 | cd "${SOURCE_DIR}" 33 | mkdir -p "${RELEASE_DIR}" 34 | 35 | project_version=$(get_pom_version) 36 | if [[ ${project_version} =~ -SNAPSHOT$ ]]; then 37 | echo "Jars should not be created for SNAPSHOT versions. Use 'update_branch_version.sh' first." 38 | exit 1 39 | fi 40 | flink_minor_version=$(echo ${FLINK_VERSION} | sed "s/.[0-9]\+$//") 41 | version=${project_version}-${flink_minor_version} 42 | 43 | echo "Deploying jars v${version} to repository.apache.org" 44 | echo "To revert this step, login to 'https://repository.apache.org' -> 'Staging repositories' -> Select repository -> 'Drop'" 45 | 46 | clone_dir=$(create_pristine_source "${SOURCE_DIR}" "${RELEASE_DIR}") 47 | cd "${clone_dir}" 48 | set_pom_version "${version}" 49 | 50 | options="-Prelease,docs-and-source -DskipTests -DretryFailedDeploymentCount=10" 51 | ${MVN} clean deploy ${options} -Dflink.version=${FLINK_VERSION} 52 | 53 | cd "${RELEASE_DIR}" 54 | rm -rf "${clone_dir}" 55 | } 56 | 57 | (deploy_staging_jars) 58 | -------------------------------------------------------------------------------- /tools/releasing/shared/stage_source_release.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | source "${SCRIPT_DIR}/_utils.sh" 24 | 25 | ########################### 26 | 27 | check_variables_set RC_NUM 28 | 29 | ########################### 30 | 31 | function create_source_release { 32 | cd ${SOURCE_DIR} 33 | mkdir -p ${RELEASE_DIR} 34 | mkdir -p ${ARTIFACTS_DIR} 35 | 36 | project=${PWD##*/} 37 | version=$(get_pom_version) 38 | if [[ ${version} =~ -SNAPSHOT$ ]]; then 39 | echo "Source releases should not be created for SNAPSHOT versions. Use 'update_branch_version.sh' first." 40 | exit 1 41 | fi 42 | 43 | echo "Creating source release v${version}" 44 | echo "To revert this step, run 'rm ${ARTIFACTS_DIR}'" 45 | 46 | clone_dir=$(create_pristine_source "${SOURCE_DIR}" "${RELEASE_DIR}") 47 | versioned_dir="${ARTIFACTS_DIR}/${project}-${version}" 48 | mv ${clone_dir} ${versioned_dir} 49 | 50 | cd "${ARTIFACTS_DIR}" 51 | tar czf ${ARTIFACTS_DIR}/${project}-${version}-src.tgz ${versioned_dir##*/} 52 | gpg --armor --detach-sig ${ARTIFACTS_DIR}/${project}-${version}-src.tgz 53 | ${SHASUM} ${project}-${version}-src.tgz >${project}-${version}-src.tgz.sha512 54 | 55 | rm -rf ${versioned_dir} 56 | } 57 | 58 | function deploy_source_release { 59 | cd ${SOURCE_DIR} 60 | project=${PWD##*/} 61 | version=$(get_pom_version)-rc${RC_NUM} 62 | 63 | release=${project}-${version} 64 | 65 | echo "Deploying source release v${version}" 66 | echo "To revert this step, run 'svn delete ${SVN_DEV_DIR}/${release}'" 67 | 68 | svn_dir=${RELEASE_DIR}/svn 69 | rm -rf ${svn_dir} 70 | mkdir -p ${svn_dir} 71 | cd ${svn_dir} 72 | 73 | svn checkout ${SVN_DEV_DIR} --depth=immediates 74 | cd flink 75 | mkdir ${release} 76 | mv ${ARTIFACTS_DIR}/* ${release} 77 | svn add ${release} 78 | svn commit -m "Add ${release}" 79 | 80 | cd ${RELEASE_DIR} 81 | rm -rf ${svn_dir} 82 | } 83 | 84 | (create_source_release) 85 | (deploy_source_release) 86 | -------------------------------------------------------------------------------- /tools/releasing/shared/update_branch_version.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$(cd -- "$(dirname -- "${BASH_SOURCE[0]}")" &>/dev/null && pwd) 21 | 22 | source "${SCRIPT_DIR}/_init.sh" 23 | source "${SCRIPT_DIR}/_utils.sh" 24 | 25 | ########################## 26 | 27 | check_variables_set NEW_VERSION 28 | 29 | ########################### 30 | 31 | function update_branch_version { 32 | cd "${SOURCE_DIR}" 33 | 34 | set_pom_version "${NEW_VERSION}" 35 | 36 | git commit -am "Update version to $NEW_VERSION" 37 | 38 | echo "Done." 39 | } 40 | 41 | (update_branch_version) 42 | -------------------------------------------------------------------------------- /tools/releasing/shared/update_japicmp_configuration.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | SCRIPT_DIR=$( cd -- "$( dirname -- "${BASH_SOURCE[0]}" )" &> /dev/null && pwd ) 21 | 22 | source ${SCRIPT_DIR}/_init.sh 23 | source ${SCRIPT_DIR}/_utils.sh 24 | 25 | ########################### 26 | 27 | check_variables_set NEW_VERSION 28 | 29 | ########################### 30 | 31 | # Idealized use-cases: 32 | # Scenario A) New major release X.0.0 33 | # Premise: 34 | # There is a main branch a version X.0-SNAPSHOT, with a japimp reference version of (X-1).Y.Z 35 | # Release flow: 36 | # - update the main to (X+1).0-SNAPSHOT, but keep the reference version intact since X.0.0 is not released (yet) 37 | # - create X.0-SNAPSHOT branch, but keep the reference version intact since X.0.0 is not released (yet) 38 | # - release X.0.0 39 | # - update the japicmp reference version of both main and X.0-SNAPSHOT to X.0.0 40 | # - enable stronger compatibility constraints for X.0-SNAPSHOT to ensure compatibility for PublicEvolving 41 | # Scenario A) New minor release X.Y.0 42 | # Premise: 43 | # There is a main branch with a version X.Y-SNAPSHOT, with a japicmp reference version of X.(Y-1).0 . 44 | # Release flow: 45 | # - update the main branch to X.(Y+1)-SNAPSHOT, but keep the reference version intact since X.Y.0 is not released (yet) 46 | # - create X.Y-SNAPSHOT branch, but keep the reference version intact since X.Y.0 is not released (yet) 47 | # - release X.Y.0 48 | # - update the japicmp reference version of both main and X.Y-SNAPSHOT to X.Y.0 49 | # - enable stronger compatibility constraints for X.Y-SNAPSHOT to ensure compatibility for PublicEvolving 50 | # Scenario C) New patch release X.Y.Z 51 | # Premise: 52 | # There is a snapshot branch with a version X.Y-SNAPSHOT, with a japicmp reference version of X.Y.(Z-1) 53 | # Release flow: 54 | # - release X.Y.Z 55 | # - update the japicmp reference version of X.Y-SNAPSHOT to X.Y.Z 56 | 57 | function enable_public_evolving_compatibility_checks() { 58 | perl -pi -e 's##${1}#' pom.xml 59 | perl -pi -e 's#\t+\@org.apache.flink.annotation.PublicEvolving.*\n##' pom.xml 60 | } 61 | 62 | function set_japicmp_reference_version() { 63 | local version=$1 64 | 65 | perl -pi -e 's#().*()#${1}'${version}'${2}#' pom.xml 66 | } 67 | 68 | function clear_exclusions() { 69 | exclusion_start=$(($(sed -n '/