Skip to content

[FLINK-37834] Flink JDBC support ClickHouse Dialect #171

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions flink-connector-jdbc-architecture/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-jdbc-clickhouse</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-jdbc-cratedb</artifactId>
Expand Down
94 changes: 94 additions & 0 deletions flink-connector-jdbc-clickhouse/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,94 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-jdbc-parent</artifactId>
<version>4.0-SNAPSHOT</version>
</parent>

<artifactId>flink-connector-jdbc-clickhouse</artifactId>
<name>Flink : Connectors : JDBC : ClickHouse</name>

<packaging>jar</packaging>

<properties>
<clickhouse.version>0.6.2</clickhouse.version>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not use the last version available 0.8.6?

</properties>

<dependencies>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-jdbc-core</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-jdbc-core</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>


<!-- ClickHouse -->
<dependency>
<groupId>com.clickhouse</groupId>
<artifactId>clickhouse-jdbc</artifactId>
<version>${clickhouse.version}</version>
<scope>provided</scope>
</dependency>

<!-- Assertions test dependencies -->

<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
<version>${assertj.version}</version>
<scope>test</scope>
</dependency>

<!-- ClickHouse tests -->
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>clickhouse</artifactId>
<scope>test</scope>
</dependency>

</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.connector.jdbc.clickhouse.database;

import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.clickhouse.database.dialect.ClickHouseDialect;
import org.apache.flink.connector.jdbc.core.database.JdbcFactory;
import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog;
import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect;

/** Factory for {@link ClickHouseDialect}. */
@Internal
public class ClickHouseFactory implements JdbcFactory {
@Override
public boolean acceptsURL(String url) {
return url.startsWith("jdbc:clickhouse:");
}

@Override
public JdbcDialect createDialect() {
return new ClickHouseDialect();
}

@Override
public JdbcCatalog createCatalog(
ClassLoader classLoader,
String catalogName,
String defaultDatabase,
String username,
String pwd,
String baseUrl) {
throw new UnsupportedOperationException("Catalog for ClickHouse is not supported yet.");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,139 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.connector.jdbc.clickhouse.database.dialect;

import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialect;
import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialectConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;

import java.util.Arrays;
import java.util.EnumSet;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;

import static java.lang.String.format;

/** JDBC dialect for ClickHouse. */
@Internal
public class ClickHouseDialect extends AbstractDialect {

private static final long serialVersionUID = 1L;

// Define MAX/MIN precision of TIMESTAMP type according to ClickHouse docs:
// https://clickhouse.com/docs/sql-reference/data-types/datetime64
private static final int MAX_TIMESTAMP_PRECISION = 9;
private static final int MIN_TIMESTAMP_PRECISION = 0;

// Define MAX/MIN precision of DECIMAL type according to ClickHouse docs:
// https://clickhouse.com/docs/sql-reference/data-types/decimal
private static final int MAX_DECIMAL_PRECISION = 76;
private static final int MIN_DECIMAL_PRECISION = 1;

@Override
public JdbcDialectConverter getRowConverter(RowType rowType) {
return new ClickHouseDialectConverter(rowType);
}

@Override
public String getLimitClause(long limit) {
return "LIMIT " + limit;
}

@Override
public Optional<String> defaultDriverName() {
return Optional.of("com.clickhouse.jdbc.ClickHouseDriver");
}

@Override
public String dialectName() {
return "ClickHouse";
}

@Override
public String quoteIdentifier(String identifier) {
return identifier;
}

// ClickHouse does not support Upsert statements
// Instead you can create a table with ReplacingMergeTree engine;
// https://clickhouse.com/docs/engines/table-engines/mergetree-family/replacingmergetree
@Override
public Optional<String> getUpsertStatement(
String tableName, String[] fieldNames, String[] uniqueKeyFields) {
return Optional.empty();
}

// ClickHouse pkField cannot be updated via sql
@Override
public String getUpdateStatement(
String tableName, String[] fieldNames, String[] conditionFields) {
String setClause =
Arrays.stream(fieldNames)
.filter(item -> !Arrays.asList(conditionFields).contains(item))
.map(f -> format("%s = :%s", quoteIdentifier(f), f))
.collect(Collectors.joining(", "));
String conditionClause =
Arrays.stream(conditionFields)
.map(f -> format("%s = :%s", quoteIdentifier(f), f))
.collect(Collectors.joining(" AND "));
return "ALTER TABLE "
+ quoteIdentifier(tableName)
+ " UPDATE "
+ setClause
+ " WHERE "
+ conditionClause;
}

@Override
public Optional<Range> decimalPrecisionRange() {
return Optional.of(Range.of(MIN_DECIMAL_PRECISION, MAX_DECIMAL_PRECISION));
}

@Override
public Optional<Range> timestampPrecisionRange() {
return Optional.of(Range.of(MIN_TIMESTAMP_PRECISION, MAX_TIMESTAMP_PRECISION));
}

@Override
public Set<LogicalTypeRoot> supportedTypes() {
// The data types used in ClickHouse are list at:
// https://clickhouse.com/docs/sql-reference/data-types

return EnumSet.of(
LogicalTypeRoot.CHAR,
LogicalTypeRoot.VARCHAR,
LogicalTypeRoot.BOOLEAN,
LogicalTypeRoot.DECIMAL,
LogicalTypeRoot.TINYINT,
LogicalTypeRoot.SMALLINT,
LogicalTypeRoot.INTEGER,
LogicalTypeRoot.BIGINT,
LogicalTypeRoot.FLOAT,
LogicalTypeRoot.DOUBLE,
LogicalTypeRoot.DATE,
LogicalTypeRoot.MAP,
LogicalTypeRoot.ARRAY,
LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE,
LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE,
LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE);
}
}
Loading