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 '/