diff --git a/api/src/main/java/org/apache/gravitino/rel/indexes/Indexes.java b/api/src/main/java/org/apache/gravitino/rel/indexes/Indexes.java index 33b8f5cd818..70d8b66a6b0 100644 --- a/api/src/main/java/org/apache/gravitino/rel/indexes/Indexes.java +++ b/api/src/main/java/org/apache/gravitino/rel/indexes/Indexes.java @@ -27,6 +27,10 @@ public class Indexes { /** MySQL does not support setting the name of the primary key, so the default name is used. */ public static final String DEFAULT_MYSQL_PRIMARY_KEY_NAME = "PRIMARY"; + /** + * ClickHouse does not support setting the name of the primary key, so the default name is used. + */ + public static final String DEFAULT_CLICKHOUSE_PRIMARY_KEY_NAME = "PRIMARY"; /** * Create a unique index on columns. Like unique (a) or unique (a, b), for complex like unique * diff --git a/build.gradle.kts b/build.gradle.kts index 401ccba3dfd..5eb2b975874 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -821,6 +821,7 @@ tasks { "catalogs:catalog-lakehouse-hudi:copyLibAndConfig", ":catalogs:catalog-jdbc-doris:copyLibAndConfig", ":catalogs:catalog-jdbc-mysql:copyLibAndConfig", + ":catalogs:catalog-jdbc-clickhouse:copyLibAndConfig", ":catalogs:catalog-jdbc-oceanbase:copyLibAndConfig", ":catalogs:catalog-jdbc-postgresql:copyLibAndConfig", ":catalogs:catalog-hadoop:copyLibAndConfig", diff --git a/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseConstants.java b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseConstants.java new file mode 100644 index 00000000000..6b6c30de13e --- /dev/null +++ b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseConstants.java @@ -0,0 +1,24 @@ +/* + * 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.gravitino.catalog.clickhouse; + +public class ClickHouseConstants { + public static final String GRAVITINO_ENGINE_KEY = "engine"; + public static final String CLICKHOUSE_ENGINE_KEY = "ENGINE"; +} diff --git a/catalogs/catalog-jdbc-clickhouse/build.gradle.kts b/catalogs/catalog-jdbc-clickhouse/build.gradle.kts new file mode 100644 index 00000000000..c6a30a3ef07 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/build.gradle.kts @@ -0,0 +1,114 @@ +/* + * 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. + */ +description = "catalog-jdbc-clickhouse" + +plugins { + `maven-publish` + id("java") + id("idea") +} + +dependencies { + implementation(project(":api")) { + exclude(group = "*") + } + implementation(project(":catalogs:catalog-common")) { + exclude(group = "*") + } + implementation(project(":catalogs:catalog-jdbc-common")) { + exclude(group = "*") + } + implementation(project(":common")) { + exclude(group = "*") + } + implementation(project(":core")) { + exclude(group = "*") + } + + implementation(libs.bundles.log4j) + implementation(libs.commons.collections4) + implementation(libs.commons.lang3) + implementation(libs.guava) + implementation(libs.httpclient5) + implementation(libs.lz4.java) + + testImplementation(project(":catalogs:catalog-jdbc-common", "testArtifacts")) + testImplementation(project(":clients:client-java")) + testImplementation(project(":integration-test-common", "testArtifacts")) + testImplementation(project(":server")) + testImplementation(project(":server-common")) + + testImplementation(libs.junit.jupiter.api) + testImplementation(libs.junit.jupiter.params) + testImplementation(libs.clickhouse.driver) + testImplementation(libs.lz4.java) + testImplementation(libs.postgresql.driver) + testImplementation(libs.testcontainers) + testImplementation(libs.testcontainers.clickhouse) + + testRuntimeOnly(libs.junit.jupiter.engine) +} + +tasks { + val runtimeJars by registering(Copy::class) { + from(configurations.runtimeClasspath) + into("build/libs") + } + + val copyCatalogLibs by registering(Copy::class) { + dependsOn("jar", "runtimeJars") + from("build/libs") { + exclude("guava-*.jar") + exclude("log4j-*.jar") + exclude("slf4j-*.jar") + } + into("$rootDir/distribution/package/catalogs/jdbc-clickhouse/libs") + } + + val copyCatalogConfig by registering(Copy::class) { + from("src/main/resources") + into("$rootDir/distribution/package/catalogs/jdbc-clickhouse/conf") + + include("jdbc-clickhouse.conf") + + exclude { details -> + details.file.isDirectory() + } + + fileMode = 0b111101101 + } + + register("copyLibAndConfig", Copy::class) { + dependsOn(copyCatalogLibs, copyCatalogConfig) + } +} + +tasks.test { + val skipITs = project.hasProperty("skipITs") + if (skipITs) { + // Exclude integration tests + exclude("**/integration/test/**") + } else { + dependsOn(tasks.jar) + } +} + +tasks.getByName("generateMetadataFileForMavenJavaPublication") { + dependsOn("runtimeJars") +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalog.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalog.java new file mode 100644 index 00000000000..f7e6a48ca3d --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalog.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.catalog.clickhouse; + +import java.util.Map; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseColumnDefaultValueConverter; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseExceptionConverter; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter; +import org.apache.gravitino.catalog.clickhouse.operation.ClickHouseDatabaseOperations; +import org.apache.gravitino.catalog.clickhouse.operation.ClickHouseTableOperations; +import org.apache.gravitino.catalog.jdbc.JdbcCatalog; +import org.apache.gravitino.catalog.jdbc.JdbcCatalogOperations; +import org.apache.gravitino.catalog.jdbc.converter.JdbcColumnDefaultValueConverter; +import org.apache.gravitino.catalog.jdbc.converter.JdbcExceptionConverter; +import org.apache.gravitino.catalog.jdbc.converter.JdbcTypeConverter; +import org.apache.gravitino.catalog.jdbc.operation.JdbcDatabaseOperations; +import org.apache.gravitino.catalog.jdbc.operation.JdbcTableOperations; +import org.apache.gravitino.connector.CatalogOperations; +import org.apache.gravitino.connector.PropertiesMetadata; +import org.apache.gravitino.connector.capability.Capability; + +/** Implementation of a Clickhouse catalog in Apache Gravitino. */ +public class ClickHouseCatalog extends JdbcCatalog { + + private static final ClickHouseTablePropertiesMetadata TABLE_PROPERTIES_META = + new ClickHouseTablePropertiesMetadata(); + + @Override + public String shortName() { + return "jdbc-clickhouse"; + } + + @Override + protected CatalogOperations newOps(Map config) { + JdbcTypeConverter jdbcTypeConverter = createJdbcTypeConverter(); + return new JdbcCatalogOperations( + createExceptionConverter(), + jdbcTypeConverter, + createJdbcDatabaseOperations(), + createJdbcTableOperations(), + createJdbcColumnDefaultValueConverter()); + } + + @Override + public Capability newCapability() { + return new ClickHouseCatalogCapability(); + } + + @Override + protected JdbcExceptionConverter createExceptionConverter() { + return new ClickHouseExceptionConverter(); + } + + @Override + protected JdbcTypeConverter createJdbcTypeConverter() { + return new ClickHouseTypeConverter(); + } + + @Override + protected JdbcDatabaseOperations createJdbcDatabaseOperations() { + return new ClickHouseDatabaseOperations(); + } + + @Override + protected JdbcTableOperations createJdbcTableOperations() { + return new ClickHouseTableOperations(); + } + + @Override + protected JdbcColumnDefaultValueConverter createJdbcColumnDefaultValueConverter() { + return new ClickHouseColumnDefaultValueConverter(); + } + + @Override + public PropertiesMetadata tablePropertiesMetadata() throws UnsupportedOperationException { + return TABLE_PROPERTIES_META; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalogCapability.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalogCapability.java new file mode 100644 index 00000000000..f4ca6e50649 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseCatalogCapability.java @@ -0,0 +1,51 @@ +/* + * 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.gravitino.catalog.clickhouse; + +import org.apache.gravitino.connector.capability.Capability; +import org.apache.gravitino.connector.capability.CapabilityResult; + +public class ClickHouseCatalogCapability implements Capability { + /** + * Regular expression explanation: ^[\w\p{L}-$/=]{1,64}$ + * + *

^ - Start of the string + * + *

[\w\p{L}-$/=]{1,64} - Consist of 1 to 64 characters of letters (both cases), digits, + * underscores, any kind of letter from any language, hyphens, dollar signs, slashes or equal + * signs + * + *

\w - matches [a-zA-Z0-9_] + * + *

\p{L} - matches any kind of letter from any language + * + *

$ - End of the string + */ + public static final String CLICKHOUSE_NAME_PATTERN = "^[\\w\\p{L}-$/=]{1,64}$"; + + @Override + public CapabilityResult specificationOnName(Scope scope, String name) { + // TODO: Validate the name against reserved words + if (!name.matches(CLICKHOUSE_NAME_PATTERN)) { + return CapabilityResult.unsupported( + String.format("The %s name '%s' is illegal.", scope, name)); + } + return CapabilityResult.SUPPORTED; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseTablePropertiesMetadata.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseTablePropertiesMetadata.java new file mode 100644 index 00000000000..ae91cde770a --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/ClickHouseTablePropertiesMetadata.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.catalog.clickhouse; + +import static org.apache.gravitino.connector.PropertyEntry.enumImmutablePropertyEntry; +import static org.apache.gravitino.connector.PropertyEntry.stringReservedPropertyEntry; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.commons.collections4.BidiMap; +import org.apache.commons.collections4.bidimap.TreeBidiMap; +import org.apache.gravitino.catalog.jdbc.JdbcTablePropertiesMetadata; +import org.apache.gravitino.connector.PropertyEntry; + +public class ClickHouseTablePropertiesMetadata extends JdbcTablePropertiesMetadata { + public static final String GRAVITINO_ENGINE_KEY = ClickHouseConstants.GRAVITINO_ENGINE_KEY; + public static final String CLICKHOUSE_ENGINE_KEY = ClickHouseConstants.CLICKHOUSE_ENGINE_KEY; + private static final Map> PROPERTIES_METADATA = + createPropertiesMetadata(); + + public static final BidiMap GRAVITINO_CONFIG_TO_CLICKHOUSE = + createGravitinoConfigToClickhouse(); + + private static BidiMap createGravitinoConfigToClickhouse() { + BidiMap map = new TreeBidiMap<>(); + map.put(GRAVITINO_ENGINE_KEY, CLICKHOUSE_ENGINE_KEY); + return map; + } + + private static Map> createPropertiesMetadata() { + Map> map = new HashMap<>(); + map.put(COMMENT_KEY, stringReservedPropertyEntry(COMMENT_KEY, "The table comment", true)); + map.put( + GRAVITINO_ENGINE_KEY, + enumImmutablePropertyEntry( + GRAVITINO_ENGINE_KEY, + "The table engine", + false, + ENGINE.class, + ENGINE.MERGETREE, + false, + false)); + return Collections.unmodifiableMap(map); + } + + /** refer https://clickhouse.com/docs/en/engines/table-engines */ + public enum ENGINE { + // MergeTree + MERGETREE("MergeTree"), + REPLACINGMERGETREE("ReplacingMergeTree"), + SUMMINGMERGETREE("SummingMergeTree"), + AGGREGATINGMERGETREE("AggregatingMergeTree"), + COLLAPSINGMERGETREE("CollapsingMergeTree"), + VERSIONEDCOLLAPSINGMERGETREE("VersionedCollapsingMergeTree"), + GRAPHITEMERGETREE("GraphiteMergeTree"), + + // Log + TINYLOG("TinyLog"), + STRIPELOG("StripeLog"), + LOG("Log"), + + // Integration Engines + ODBC("ODBC"), + JDBC("JDBC"), + MySQL("MySQL"), + MONGODB("MongoDB"), + Redis("Redis"), + HDFS("HDFS"), + S3("S3"), + KAFKA("Kafka"), + EMBEDDEDROCKSDB("EmbeddedRocksDB"), + RABBITMQ("RabbitMQ"), + POSTGRESQL("PostgreSQL"), + S3QUEUE("S3Queue"), + TIMESERIES("TimeSeries"), + + // Special Engines + DISTRIBUTED("Distributed"), + DICTIONARY("Dictionary"), + MERGE("Merge"), + FILE("File"), + NULL("Null"), + SET("Set"), + JOIN("Join"), + URL("URL"), + VIEW("View"), + MEMORY("Memory"), + BUFFER("Buffer"), + KEEPER_MAP("KeeperMap"); + + private final String value; + + ENGINE(String value) { + this.value = value; + } + + public String getValue() { + return this.value; + } + } + + @Override + protected Map> specificPropertyEntries() { + return PROPERTIES_METADATA; + } + + @Override + public Map transformToJdbcProperties(Map properties) { + return Collections.unmodifiableMap( + new HashMap() { + { + properties.forEach( + (key, value) -> { + if (GRAVITINO_CONFIG_TO_CLICKHOUSE.containsKey(key)) { + put(GRAVITINO_CONFIG_TO_CLICKHOUSE.get(key), value); + } + }); + } + }); + } + + @Override + public Map convertFromJdbcProperties(Map properties) { + BidiMap clickhouseConfigToGravitino = + GRAVITINO_CONFIG_TO_CLICKHOUSE.inverseBidiMap(); + return Collections.unmodifiableMap( + new HashMap() { + { + properties.forEach( + (key, value) -> { + if (clickhouseConfigToGravitino.containsKey(key)) { + put(clickhouseConfigToGravitino.get(key), value); + } + }); + } + }); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseColumnDefaultValueConverter.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseColumnDefaultValueConverter.java new file mode 100644 index 00000000000..8b7ce4b41c0 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseColumnDefaultValueConverter.java @@ -0,0 +1,164 @@ +/* + * 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.gravitino.catalog.clickhouse.converter; + +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_NOT_SET; +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_OF_CURRENT_TIMESTAMP; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import org.apache.gravitino.catalog.jdbc.converter.JdbcColumnDefaultValueConverter; +import org.apache.gravitino.catalog.jdbc.converter.JdbcTypeConverter; +import org.apache.gravitino.rel.expressions.Expression; +import org.apache.gravitino.rel.expressions.FunctionExpression; +import org.apache.gravitino.rel.expressions.UnparsedExpression; +import org.apache.gravitino.rel.expressions.literals.Literal; +import org.apache.gravitino.rel.expressions.literals.Literals; +import org.apache.gravitino.rel.types.Decimal; +import org.apache.gravitino.rel.types.Type; +import org.apache.gravitino.rel.types.Types; + +public class ClickHouseColumnDefaultValueConverter extends JdbcColumnDefaultValueConverter { + + protected static final String NOW = "now"; + Expression DEFAULT_VALUE_OF_NOW = FunctionExpression.of("now"); + + public String fromGravitino(Expression defaultValue) { + if (DEFAULT_VALUE_NOT_SET.equals(defaultValue)) { + return null; + } + + if (defaultValue instanceof FunctionExpression) { + FunctionExpression functionExpression = (FunctionExpression) defaultValue; + return String.format("(%s)", functionExpression); + } + + if (defaultValue instanceof Literal) { + Literal literal = (Literal) defaultValue; + Type type = literal.dataType(); + if (defaultValue.equals(Literals.NULL)) { + return NULL; + } else if (type instanceof Type.NumericType) { + return literal.value().toString(); + } else { + Object value = literal.value(); + if (value instanceof LocalDateTime) { + value = ((LocalDateTime) value).format(DATE_TIME_FORMATTER); + } + return String.format("'%s'", value); + } + } + + throw new IllegalArgumentException("Not a supported column default value: " + defaultValue); + } + + @Override + public Expression toGravitino( + JdbcTypeConverter.JdbcTypeBean type, + String columnDefaultValue, + boolean isExpression, + boolean nullable) { + if (columnDefaultValue == null || columnDefaultValue.isEmpty()) { + return nullable ? Literals.NULL : DEFAULT_VALUE_NOT_SET; + } + + String reallyType = type.getTypeName(); + if (reallyType.startsWith("Nullable(")) { + reallyType = type.getTypeName().substring(9, type.getTypeName().length() - 1); + } + + if (reallyType.startsWith("Decimal(")) { + reallyType = "Decimal"; + } + + if (reallyType.startsWith("FixedString(")) { + reallyType = "FixedString"; + } + + if (nullable) { + if (columnDefaultValue.equals("NULL")) { + return Literals.NULL; + } + } + + // TODO clickhouse has bug which isExpression is false when is really expression + if (isExpression) { + if (columnDefaultValue.equals(NOW)) { + return DEFAULT_VALUE_OF_NOW; + } + // The parsing of ClickHouse expressions is complex, so we are not currently undertaking the + // parsing. + return UnparsedExpression.of(columnDefaultValue); + } + + // need exclude begin and end "'" + String reallyValue = + columnDefaultValue.startsWith("'") + ? columnDefaultValue.substring(1, columnDefaultValue.length() - 1) + : columnDefaultValue; + + try { + switch (reallyType) { + case ClickHouseTypeConverter.INT8: + return Literals.byteLiteral(Byte.valueOf(reallyValue)); + case ClickHouseTypeConverter.UINT8: + return Literals.unsignedByteLiteral(Short.valueOf(reallyValue)); + case ClickHouseTypeConverter.INT16: + return Literals.shortLiteral(Short.valueOf(reallyValue)); + case ClickHouseTypeConverter.UINT16: + return Literals.unsignedShortLiteral(Integer.valueOf(reallyValue)); + case ClickHouseTypeConverter.INT32: + return Literals.integerLiteral(Integer.valueOf(reallyValue)); + case ClickHouseTypeConverter.UINT32: + return Literals.unsignedIntegerLiteral(Long.valueOf(reallyValue)); + case ClickHouseTypeConverter.INT64: + return Literals.longLiteral(Long.valueOf(reallyValue)); + case ClickHouseTypeConverter.UINT64: + return Literals.unsignedLongLiteral(Decimal.of(reallyValue)); + case ClickHouseTypeConverter.FLOAT32: + return Literals.floatLiteral(Float.valueOf(reallyValue)); + case ClickHouseTypeConverter.FLOAT64: + return Literals.doubleLiteral(Double.valueOf(reallyValue)); + case ClickHouseTypeConverter.DECIMAL: + if (reallyValue.equals("0.")) { + reallyValue = "0.0"; + } + return Literals.decimalLiteral( + Decimal.of(reallyValue, type.getColumnSize(), type.getScale())); + case ClickHouseTypeConverter.DATE: + if (reallyValue.equals("")) { + return Literals.NULL; + } + return Literals.dateLiteral(LocalDate.parse(reallyValue, DATE_FORMATTER)); + case ClickHouseTypeConverter.DATETIME: + return CURRENT_TIMESTAMP.equals(reallyValue) + ? DEFAULT_VALUE_OF_CURRENT_TIMESTAMP + : Literals.timestampLiteral(LocalDateTime.parse(reallyValue, DATE_TIME_FORMATTER)); + case ClickHouseTypeConverter.STRING: + return Literals.of(reallyValue, Types.StringType.get()); + case ClickHouseTypeConverter.FIXEDSTRING: + return Literals.of(reallyValue, Types.FixedCharType.of(type.getColumnSize())); + default: + return UnparsedExpression.of(reallyValue); + } + } catch (Exception ex) { + return UnparsedExpression.of(reallyValue); + } + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseExceptionConverter.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseExceptionConverter.java new file mode 100644 index 00000000000..ea0cf617a8d --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseExceptionConverter.java @@ -0,0 +1,65 @@ +/* + * 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.gravitino.catalog.clickhouse.converter; + +import java.sql.SQLException; +import org.apache.gravitino.catalog.jdbc.converter.JdbcExceptionConverter; +import org.apache.gravitino.exceptions.GravitinoRuntimeException; +import org.apache.gravitino.exceptions.NoSuchSchemaException; +import org.apache.gravitino.exceptions.NoSuchTableException; +import org.apache.gravitino.exceptions.SchemaAlreadyExistsException; +import org.apache.gravitino.exceptions.TableAlreadyExistsException; + +/** Exception converter to Apache Gravitino exception for ClickHouse. */ +public class ClickHouseExceptionConverter extends JdbcExceptionConverter { + + // see: SELECT concat('\t', name, ' = ', toString(number)) + // FROM + // ( + // SELECT + // number, + // errorCodeToName(number) AS name + // FROM system.numbers + // LIMIT 2000 + // ) + // WHERE NOT empty(errorCodeToName(number)) + static final int UNKNOWN_DATABASE = 81; + static final int DATABASE_ALREADY_EXISTS = 82; + + static final int TABLE_ALREADY_EXISTS = 57; + static final int TABLE_IS_DROPPED = 218; + + @SuppressWarnings("FormatStringAnnotation") + @Override + public GravitinoRuntimeException toGravitinoException(SQLException sqlException) { + int errorCode = sqlException.getErrorCode(); + switch (errorCode) { + case DATABASE_ALREADY_EXISTS: + return new SchemaAlreadyExistsException(sqlException, sqlException.getMessage()); + case TABLE_ALREADY_EXISTS: + return new TableAlreadyExistsException(sqlException, sqlException.getMessage()); + case UNKNOWN_DATABASE: + return new NoSuchSchemaException(sqlException, sqlException.getMessage()); + case TABLE_IS_DROPPED: + return new NoSuchTableException(sqlException, sqlException.getMessage()); + default: + return new GravitinoRuntimeException(sqlException, sqlException.getMessage()); + } + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseTypeConverter.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseTypeConverter.java new file mode 100644 index 00000000000..c116018869e --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseTypeConverter.java @@ -0,0 +1,193 @@ +/* + * 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.gravitino.catalog.clickhouse.converter; + +import org.apache.gravitino.catalog.jdbc.converter.JdbcTypeConverter; +import org.apache.gravitino.rel.types.Type; +import org.apache.gravitino.rel.types.Types; + +/** Type converter for ClickHouse. */ +public class ClickHouseTypeConverter extends JdbcTypeConverter { + + static final String INT8 = "Int8"; + static final String INT16 = "Int16"; + static final String INT32 = "Int32"; + static final String INT64 = "Int64"; + static final String INT128 = "Int128"; + static final String INT256 = "Int256"; + static final String UINT8 = "UInt8"; + static final String UINT16 = "UInt16"; + static final String UINT32 = "UInt32"; + static final String UINT64 = "UInt64"; + static final String UINT128 = "UInt128"; + static final String UINT256 = "UInt256"; + + static final String FLOAT32 = "Float32"; + static final String FLOAT64 = "Float64"; + static final String BFLOAT16 = "BFloat16"; + static final String DECIMAL = "Decimal"; + static final String STRING = "String"; + static final String FIXEDSTRING = "FixedString"; + static final String DATE = "Date"; + static final String DATE32 = "Date32"; + static final String DATETIME = "DateTime"; + static final String DATETIME64 = "DateTime64"; + static final String ENUM = "Enum"; + static final String BOOL = "Bool"; + static final String UUID = "UUID"; + + // bellow is Object Data Type + static final String IPV4 = "IPv4"; + static final String IPV6 = "IPv6"; + static final String ARRAY = "Array"; + static final String TUPLE = "Tuple"; + static final String MAP = "Map"; + static final String VARIANT = "Variant"; + static final String LOWCARDINALITY = "LowCardinality"; + static final String NULLABLE = "Nullable"; + static final String AGGREGATEFUNCTION = "AggregateFunction"; + static final String SIMPLEAGGREGATEFUNCTION = "SimpleAggregateFunction"; + static final String GEO = "Geo"; + + // bellow is Special Data Types + static final String Domains = "Domains"; + static final String Nested = "Nested"; + static final String Dynamic = "Dynamic"; + static final String JSON = "JSON"; + + @Override + public Type toGravitino(JdbcTypeBean typeBean) { + String typeName = typeBean.getTypeName(); + if (typeName.startsWith("Nullable(")) { + typeName = typeName.substring(9, typeName.length() - 1); + } + + if (typeName.startsWith("Decimal(")) { + typeName = "Decimal"; + } + + if (typeName.startsWith("FixedString(")) { + typeName = "FixedString"; + } + + switch (typeName) { + case INT8: + return Types.ByteType.get(); + case INT16: + return Types.ShortType.get(); + case INT32: + return Types.IntegerType.get(); + case INT64: + return Types.LongType.get(); + case UINT8: + return Types.ByteType.unsigned(); + case UINT16: + return Types.ShortType.unsigned(); + case UINT32: + return Types.IntegerType.unsigned(); + case UINT64: + return Types.LongType.unsigned(); + case FLOAT32: + return Types.FloatType.get(); + case FLOAT64: + return Types.DoubleType.get(); + case DECIMAL: + return Types.DecimalType.of(typeBean.getColumnSize(), typeBean.getScale()); + case STRING: + return Types.StringType.get(); + case FIXEDSTRING: + return Types.FixedCharType.of(typeBean.getColumnSize()); + case DATE: + return Types.DateType.get(); + case DATE32: + return Types.DateType.get(); + case DATETIME: + return Types.TimestampType.withoutTimeZone(); + case DATETIME64: + return Types.TimestampType.withoutTimeZone(); + case BOOL: + return Types.BooleanType.get(); + case UUID: + return Types.UUIDType.get(); + default: + return Types.ExternalType.of(typeBean.getTypeName()); + } + } + + @Override + public String fromGravitino(Type type) { + if (type instanceof Types.ByteType) { + if (((Types.ByteType) type).signed()) { + return INT8; + } else { + return UINT8; + } + } else if (type instanceof Types.ShortType) { + if (((Types.ShortType) type).signed()) { + return INT16; + } else { + return UINT16; + } + } else if (type instanceof Types.IntegerType) { + if (((Types.IntegerType) type).signed()) { + return INT32; + } else { + return UINT32; + } + } else if (type instanceof Types.LongType) { + if (((Types.LongType) type).signed()) { + return INT64; + } else { + return UINT64; + } + } else if (type instanceof Types.FloatType) { + return FLOAT32; + } else if (type instanceof Types.DoubleType) { + return FLOAT64; + } else if (type instanceof Types.StringType) { + return STRING; + } else if (type instanceof Types.DateType) { + return DATE; + } else if (type instanceof Types.TimestampType) { + return DATETIME; + } else if (type instanceof Types.TimeType) { + return INT64; + } else if (type instanceof Types.DecimalType) { + return DECIMAL + + "(" + + ((Types.DecimalType) type).precision() + + "," + + ((Types.DecimalType) type).scale() + + ")"; + } else if (type instanceof Types.VarCharType) { + return STRING; + } else if (type instanceof Types.FixedCharType) { + return FIXEDSTRING + "(" + ((Types.FixedCharType) type).length() + ")"; + } else if (type instanceof Types.BooleanType) { + return BOOL; + } else if (type instanceof Types.UUIDType) { + return UUID; + } else if (type instanceof Types.ExternalType) { + return ((Types.ExternalType) type).catalogString(); + } + throw new IllegalArgumentException( + String.format( + "Couldn't convert Gravitino type %s to ClickHouse type", type.simpleString())); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseDatabaseOperations.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseDatabaseOperations.java new file mode 100644 index 00000000000..04153cb20a4 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseDatabaseOperations.java @@ -0,0 +1,66 @@ +/* + * 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.gravitino.catalog.clickhouse.operation; + +import com.google.common.collect.ImmutableSet; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import org.apache.gravitino.catalog.jdbc.operation.JdbcDatabaseOperations; + +/** Database operations for ClickHouse. */ +public class ClickHouseDatabaseOperations extends JdbcDatabaseOperations { + + @Override + protected boolean supportSchemaComment() { + return false; + } + + @Override + protected Set createSysDatabaseNameSet() { + return ImmutableSet.of("information_schema", "INFORMATION_SCHEMA", "default", "system"); + } + + @Override + public List listDatabases() { + List databaseNames = new ArrayList<>(); + try (final Connection connection = getConnection()) { + // It is possible that other catalogs have been deleted, + // causing the following statement to error, + // so here we manually set a system catalog + connection.setCatalog(createSysDatabaseNameSet().iterator().next()); + try (Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery("SHOW DATABASES")) { + while (resultSet.next()) { + String databaseName = resultSet.getString(1); + if (!isSystemDatabase(databaseName)) { + databaseNames.add(databaseName); + } + } + } + return databaseNames; + } catch (final SQLException se) { + throw this.exceptionMapper.toGravitinoException(se); + } + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseTableOperations.java b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseTableOperations.java new file mode 100644 index 00000000000..0571f3339b8 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/java/org/apache/gravitino/catalog/clickhouse/operation/ClickHouseTableOperations.java @@ -0,0 +1,730 @@ +/* + * 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.gravitino.catalog.clickhouse.operation; + +import static org.apache.gravitino.catalog.clickhouse.ClickHouseTablePropertiesMetadata.CLICKHOUSE_ENGINE_KEY; +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_NOT_SET; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.BooleanUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.StringIdentifier; +import org.apache.gravitino.catalog.jdbc.JdbcColumn; +import org.apache.gravitino.catalog.jdbc.JdbcTable; +import org.apache.gravitino.catalog.jdbc.operation.JdbcTableOperations; +import org.apache.gravitino.catalog.jdbc.utils.JdbcConnectorUtils; +import org.apache.gravitino.exceptions.NoSuchColumnException; +import org.apache.gravitino.exceptions.NoSuchTableException; +import org.apache.gravitino.rel.Column; +import org.apache.gravitino.rel.TableChange; +import org.apache.gravitino.rel.expressions.distributions.Distribution; +import org.apache.gravitino.rel.expressions.distributions.Distributions; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; +import org.apache.gravitino.rel.expressions.transforms.Transform; +import org.apache.gravitino.rel.indexes.Index; +import org.apache.gravitino.rel.indexes.Indexes; +import org.apache.gravitino.rel.types.Types; + +/** Table operations for ClickHouse. */ +public class ClickHouseTableOperations extends JdbcTableOperations { + + private static final String BACK_QUOTE = "`"; + private static final String CLICKHOUSE_AUTO_INCREMENT = "AUTO_INCREMENT"; + private static final String CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG = + "Clickhouse does not support nested column names."; + + /** + * ClickHouse does not support some multiple changes in one statement, So rewrite this method, one + * by one to apply TableChange to the table. + * + * @param databaseName The name of the database. + * @param tableName The name of the table. + * @param changes The changes to apply to the table. + */ + @Override + public void alterTable(String databaseName, String tableName, TableChange... changes) + throws NoSuchTableException { + LOG.info("Attempting to alter table {} from database {}", tableName, databaseName); + try (Connection connection = getConnection(databaseName)) { + for (TableChange change : changes) { + String sql = generateAlterTableSql(databaseName, tableName, change); + if (StringUtils.isEmpty(sql)) { + LOG.info("No changes to alter table {} from database {}", tableName, databaseName); + return; + } + JdbcConnectorUtils.executeUpdate(connection, sql); + } + LOG.info("Alter table {} from database {}", tableName, databaseName); + } catch (final SQLException se) { + throw this.exceptionMapper.toGravitinoException(se); + } + } + + @Override + protected List getIndexes(Connection connection, String databaseName, String tableName) + throws SQLException { + // cause clickhouse not impl getPrimaryKeys yet, ref: + // https://github.com/ClickHouse/clickhouse-java/issues/1625 + String sql = + "SELECT NULL AS TABLE_CAT, " + + "system.tables.database AS TABLE_SCHEM, " + + "system.tables.name AS TABLE_NAME, " + + "trim(c.1) AS COLUMN_NAME, " + + "c.2 AS KEY_SEQ, " + + "'PRIMARY' AS PK_NAME " + + "FROM system.tables " + + "ARRAY JOIN arrayZip(splitByChar(',', primary_key), arrayEnumerate(splitByChar(',', primary_key))) as c " + + "WHERE system.tables.primary_key <> '' " + + "AND system.tables.database = '" + + databaseName + + "' " + + "AND system.tables.name = '" + + tableName + + "' " + + "ORDER BY COLUMN_NAME"; + + try (PreparedStatement preparedStatement = connection.prepareStatement(sql); + ResultSet resultSet = preparedStatement.executeQuery()) { + + List indexes = new ArrayList<>(); + while (resultSet.next()) { + String indexName = resultSet.getString("PK_NAME"); + String columnName = resultSet.getString("COLUMN_NAME"); + indexes.add( + Indexes.of(Index.IndexType.PRIMARY_KEY, indexName, new String[][] {{columnName}})); + } + return indexes; + } catch (SQLException e) { + throw exceptionMapper.toGravitinoException(e); + } + } + + @Override + protected String generateCreateTableSql( + String tableName, + JdbcColumn[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + Index[] indexes) { + throw new UnsupportedOperationException( + "generateCreateTableSql with out sortOrders in clickhouse is not supported"); + } + + @Override + protected String generateCreateTableSql( + String tableName, + JdbcColumn[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + Index[] indexes, + SortOrder[] sortOrders) { + if (ArrayUtils.isNotEmpty(partitioning)) { + throw new UnsupportedOperationException( + "Currently we do not support Partitioning in clickhouse"); + } + + Preconditions.checkArgument( + Distributions.NONE.equals(distribution), "ClickHouse does not support distribution"); + + validateIncrementCol(columns, indexes); + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder + .append("CREATE TABLE ") + .append(BACK_QUOTE) + .append(tableName) + .append(BACK_QUOTE) + .append(" (\n"); + + // Add columns + for (int i = 0; i < columns.length; i++) { + JdbcColumn column = columns[i]; + sqlBuilder + .append(SPACE) + .append(SPACE) + .append(BACK_QUOTE) + .append(column.name()) + .append(BACK_QUOTE); + + appendColumnDefinition(column, sqlBuilder); + // Add a comma for the next column, unless it's the last one + if (i < columns.length - 1) { + sqlBuilder.append(",\n"); + } + } + + appendIndexesSql(indexes, sqlBuilder); + + sqlBuilder.append("\n)"); + + // Add table properties if any + if (MapUtils.isNotEmpty(properties)) { + sqlBuilder.append( + properties.entrySet().stream() + .map(entry -> String.format("%s = %s", entry.getKey(), entry.getValue())) + .collect(Collectors.joining(",\n", "\n", ""))); + } + + if (ArrayUtils.isNotEmpty(sortOrders)) { + if (sortOrders.length > 1) { + throw new UnsupportedOperationException( + "Currently we do not support sortOrders's length > 1"); + } else if (sortOrders[0].nullOrdering() != null || sortOrders[0].direction() != null) { + // If no value is set earlier, some default values will be set. + // It is difficult to determine whether the user has set a value. + LOG.warn( + "clickhouse currently do not support nullOrdering: {} and direction: {} of sortOrders,and will ignore these", + sortOrders[0].nullOrdering(), + sortOrders[0].direction()); + } + sqlBuilder.append( + " \n ORDER BY " + BACK_QUOTE + sortOrders[0].expression() + BACK_QUOTE + " \n"); + } + + // Add table comment if specified + if (StringUtils.isNotEmpty(comment)) { + sqlBuilder.append(" COMMENT '").append(comment).append("'"); + } + + // Return the generated SQL statement + String result = sqlBuilder.toString(); + + LOG.info("Generated create table:{} sql: {}", tableName, result); + return result; + } + + public static void appendIndexesSql(Index[] indexes, StringBuilder sqlBuilder) { + if (indexes == null) { + return; + } + + for (Index index : indexes) { + String fieldStr = getIndexFieldStr(index.fieldNames()); + sqlBuilder.append(",\n"); + switch (index.type()) { + case PRIMARY_KEY: + if (null != index.name() + && !StringUtils.equalsIgnoreCase( + index.name(), Indexes.DEFAULT_CLICKHOUSE_PRIMARY_KEY_NAME)) { + LOG.warn( + "Primary key name must be PRIMARY in ClickHouse, the name {} will be ignored.", + index.name()); + } + sqlBuilder.append(" PRIMARY KEY (").append(fieldStr).append(")"); + break; + case UNIQUE_KEY: + throw new IllegalArgumentException( + "Gravitino clickHouse doesn't support index : " + index.type()); + default: + throw new IllegalArgumentException( + "Gravitino Clickhouse doesn't support index : " + index.type()); + } + } + } + + @Override + protected boolean getAutoIncrementInfo(ResultSet resultSet) throws SQLException { + return "YES".equalsIgnoreCase(resultSet.getString("IS_AUTOINCREMENT")); + } + + @Override + protected Map getTableProperties(Connection connection, String tableName) + throws SQLException { + try (PreparedStatement statement = + connection.prepareStatement("select * from system.tables where name = ? ")) { + statement.setString(1, tableName); + try (ResultSet resultSet = statement.executeQuery()) { + while (resultSet.next()) { + String name = resultSet.getString("name"); + if (Objects.equals(name, tableName)) { + return Collections.unmodifiableMap( + new HashMap() { + { + put(COMMENT, resultSet.getString(COMMENT)); + put(CLICKHOUSE_ENGINE_KEY, resultSet.getString(CLICKHOUSE_ENGINE_KEY)); + } + }); + } + } + + throw new NoSuchTableException( + "Table %s does not exist in %s.", tableName, connection.getCatalog()); + } + } + } + + protected ResultSet getTables(Connection connection) throws SQLException { + final DatabaseMetaData metaData = connection.getMetaData(); + String catalogName = connection.getCatalog(); + String schemaName = connection.getSchema(); + // CK tables include : DICTIONARY", "LOG TABLE", "MEMORY TABLE", + // "REMOTE TABLE", "TABLE", "VIEW", "SYSTEM TABLE", "TEMPORARY TABLE + return metaData.getTables(catalogName, schemaName, null, null); + } + + @Override + protected void correctJdbcTableFields( + Connection connection, String databaseName, String tableName, JdbcTable.Builder tableBuilder) + throws SQLException { + if (StringUtils.isEmpty(tableBuilder.comment())) { + // In Clickhouse version 5.7, the comment field value cannot be obtained in the driver API. + LOG.warn("Not found comment in clickhouse driver api. Will try to get comment from sql"); + tableBuilder.withComment( + tableBuilder.properties().getOrDefault(COMMENT, tableBuilder.comment())); + } + } + + @Override + protected String generatePurgeTableSql(String tableName) { + throw new UnsupportedOperationException( + "ClickHouse does not support purge table in Gravitino, please use drop table"); + } + + @Override + protected String generateAlterTableSql( + String databaseName, String tableName, TableChange... changes) { + // Not all operations require the original table information, so lazy loading is used here + JdbcTable lazyLoadTable = null; + TableChange.UpdateComment updateComment = null; + List setProperties = new ArrayList<>(); + List alterSql = new ArrayList<>(); + for (int i = 0; i < changes.length; i++) { + TableChange change = changes[i]; + if (change instanceof TableChange.UpdateComment) { + updateComment = (TableChange.UpdateComment) change; + } else if (change instanceof TableChange.SetProperty) { + // The set attribute needs to be added at the end. + setProperties.add(((TableChange.SetProperty) change)); + } else if (change instanceof TableChange.RemoveProperty) { + // clickhouse does not support deleting table attributes, it can be replaced by Set Property + throw new IllegalArgumentException("Remove property is not supported yet"); + } else if (change instanceof TableChange.AddColumn) { + TableChange.AddColumn addColumn = (TableChange.AddColumn) change; + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + alterSql.add(addColumnFieldDefinition(addColumn)); + } else if (change instanceof TableChange.RenameColumn) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + TableChange.RenameColumn renameColumn = (TableChange.RenameColumn) change; + alterSql.add(renameColumnFieldDefinition(renameColumn)); + } else if (change instanceof TableChange.UpdateColumnDefaultValue) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + TableChange.UpdateColumnDefaultValue updateColumnDefaultValue = + (TableChange.UpdateColumnDefaultValue) change; + alterSql.add( + updateColumnDefaultValueFieldDefinition(updateColumnDefaultValue, lazyLoadTable)); + } else if (change instanceof TableChange.UpdateColumnType) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + TableChange.UpdateColumnType updateColumnType = (TableChange.UpdateColumnType) change; + alterSql.add(updateColumnTypeFieldDefinition(updateColumnType, lazyLoadTable)); + } else if (change instanceof TableChange.UpdateColumnComment) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + TableChange.UpdateColumnComment updateColumnComment = + (TableChange.UpdateColumnComment) change; + alterSql.add(updateColumnCommentFieldDefinition(updateColumnComment, lazyLoadTable)); + } else if (change instanceof TableChange.UpdateColumnPosition) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + TableChange.UpdateColumnPosition updateColumnPosition = + (TableChange.UpdateColumnPosition) change; + alterSql.add(updateColumnPositionFieldDefinition(updateColumnPosition, lazyLoadTable)); + } else if (change instanceof TableChange.DeleteColumn) { + TableChange.DeleteColumn deleteColumn = (TableChange.DeleteColumn) change; + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + String deleteColSql = deleteColumnFieldDefinition(deleteColumn, lazyLoadTable); + if (StringUtils.isNotEmpty(deleteColSql)) { + alterSql.add(deleteColSql); + } + } else if (change instanceof TableChange.UpdateColumnNullability) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + alterSql.add( + updateColumnNullabilityDefinition( + (TableChange.UpdateColumnNullability) change, lazyLoadTable)); + } else if (change instanceof TableChange.DeleteIndex) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + alterSql.add(deleteIndexDefinition(lazyLoadTable, (TableChange.DeleteIndex) change)); + } else if (change instanceof TableChange.UpdateColumnAutoIncrement) { + lazyLoadTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + alterSql.add( + updateColumnAutoIncrementDefinition( + lazyLoadTable, (TableChange.UpdateColumnAutoIncrement) change)); + } else { + throw new IllegalArgumentException( + "Unsupported table change type: " + change.getClass().getName()); + } + } + if (!setProperties.isEmpty()) { + alterSql.add(generateAlterTableProperties(setProperties)); + } + + // Last modified comment + if (null != updateComment) { + String newComment = updateComment.getNewComment(); + if (null == StringIdentifier.fromComment(newComment)) { + // Detect and add Gravitino id. + JdbcTable jdbcTable = getOrCreateTable(databaseName, tableName, lazyLoadTable); + StringIdentifier identifier = StringIdentifier.fromComment(jdbcTable.comment()); + if (null != identifier) { + newComment = StringIdentifier.addToComment(identifier, newComment); + } + } + alterSql.add(" MODIFY COMMENT '" + newComment + "'"); + } + + if (!setProperties.isEmpty()) { + alterSql.add(generateAlterTableProperties(setProperties)); + } + + if (CollectionUtils.isEmpty(alterSql)) { + return ""; + } + // Return the generated SQL statement + String result = "ALTER TABLE `" + tableName + "`\n" + String.join(",\n", alterSql) + ";"; + LOG.info("Generated alter table:{} sql: {}", databaseName + "." + tableName, result); + return result; + } + + private String updateColumnAutoIncrementDefinition( + JdbcTable table, TableChange.UpdateColumnAutoIncrement change) { + if (change.fieldName().length > 1) { + throw new UnsupportedOperationException("Nested column names are not supported"); + } + String col = change.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(table, col); + if (change.isAutoIncrement()) { + Preconditions.checkArgument( + Types.allowAutoIncrement(column.dataType()), + "Auto increment is not allowed, type: " + column.dataType()); + } + JdbcColumn updateColumn = + JdbcColumn.builder() + .withName(col) + .withDefaultValue(column.defaultValue()) + .withNullable(column.nullable()) + .withType(column.dataType()) + .withComment(column.comment()) + .withAutoIncrement(change.isAutoIncrement()) + .build(); + return MODIFY_COLUMN + + BACK_QUOTE + + col + + BACK_QUOTE + + appendColumnDefinition(updateColumn, new StringBuilder()); + } + + @VisibleForTesting + static String deleteIndexDefinition( + JdbcTable lazyLoadTable, TableChange.DeleteIndex deleteIndex) { + if (deleteIndex.isIfExists()) { + if (Arrays.stream(lazyLoadTable.index()) + .anyMatch(index -> index.name().equals(deleteIndex.getName()))) { + throw new IllegalArgumentException("Index does not exist"); + } + } + return "DROP INDEX " + BACK_QUOTE + deleteIndex.getName() + BACK_QUOTE; + } + + private String updateColumnNullabilityDefinition( + TableChange.UpdateColumnNullability change, JdbcTable table) { + validateUpdateColumnNullable(change, table); + String col = change.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(table, col); + JdbcColumn updateColumn = + JdbcColumn.builder() + .withName(col) + .withDefaultValue(column.defaultValue()) + .withNullable(change.nullable()) + .withType(column.dataType()) + .withComment(column.comment()) + .withAutoIncrement(column.autoIncrement()) + .build(); + return MODIFY_COLUMN + + BACK_QUOTE + + col + + BACK_QUOTE + + appendColumnDefinition(updateColumn, new StringBuilder()); + } + + private String generateAlterTableProperties(List setProperties) { + if (CollectionUtils.isNotEmpty(setProperties)) { + throw new UnsupportedOperationException("alter table properties in ck is not supported"); + } + + return ""; + // return setProperties.stream() + // .map( + // setProperty -> + // String.format("%s = %s", setProperty.getProperty(), setProperty.getValue())) + // .collect(Collectors.joining(",\n")); + } + + private String updateColumnCommentFieldDefinition( + TableChange.UpdateColumnComment updateColumnComment, JdbcTable jdbcTable) { + String newComment = updateColumnComment.getNewComment(); + if (updateColumnComment.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = updateColumnComment.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(jdbcTable, col); + JdbcColumn updateColumn = + JdbcColumn.builder() + .withName(col) + .withDefaultValue(column.defaultValue()) + .withNullable(column.nullable()) + .withType(column.dataType()) + .withComment(newComment) + .withAutoIncrement(column.autoIncrement()) + .build(); + return MODIFY_COLUMN + + BACK_QUOTE + + col + + BACK_QUOTE + + appendColumnDefinition(updateColumn, new StringBuilder()); + } + + private String addColumnFieldDefinition(TableChange.AddColumn addColumn) { + String dataType = typeConverter.fromGravitino(addColumn.getDataType()); + if (addColumn.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = addColumn.fieldName()[0]; + + StringBuilder columnDefinition = new StringBuilder(); + // [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after | FIRST] + if (!addColumn.isNullable()) { + columnDefinition + .append("ADD COLUMN ") + .append(BACK_QUOTE) + .append(col) + .append(BACK_QUOTE) + .append(SPACE) + .append(dataType) + .append(SPACE); + } else { + columnDefinition + .append("ADD COLUMN ") + .append(BACK_QUOTE) + .append(col) + .append(BACK_QUOTE) + .append(SPACE) + .append("Nullable(") + .append(dataType) + .append(")") + .append(SPACE); + } + + if (addColumn.isAutoIncrement()) { + Preconditions.checkArgument( + Types.allowAutoIncrement(addColumn.getDataType()), + "Auto increment is not allowed, type: " + addColumn.getDataType()); + columnDefinition.append(CLICKHOUSE_AUTO_INCREMENT).append(SPACE); + } + + // Append default value if available + if (!Column.DEFAULT_VALUE_NOT_SET.equals(addColumn.getDefaultValue())) { + columnDefinition + .append("DEFAULT ") + .append(columnDefaultValueConverter.fromGravitino(addColumn.getDefaultValue())) + .append(SPACE); + } + + // Append comment if available after default value + if (StringUtils.isNotEmpty(addColumn.getComment())) { + columnDefinition.append("COMMENT '").append(addColumn.getComment()).append("' "); + } + + // Append position if available + if (addColumn.getPosition() instanceof TableChange.First) { + columnDefinition.append("FIRST"); + } else if (addColumn.getPosition() instanceof TableChange.After) { + TableChange.After afterPosition = (TableChange.After) addColumn.getPosition(); + columnDefinition + .append(AFTER) + .append(BACK_QUOTE) + .append(afterPosition.getColumn()) + .append(BACK_QUOTE); + } else if (addColumn.getPosition() instanceof TableChange.Default) { + // do nothing, follow the default behavior of clickhouse + } else { + throw new IllegalArgumentException("Invalid column position."); + } + + return columnDefinition.toString(); + } + + private String renameColumnFieldDefinition(TableChange.RenameColumn renameColumn) { + if (renameColumn.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + + String oldColumnName = renameColumn.fieldName()[0]; + String newColumnName = renameColumn.getNewName(); + StringBuilder sqlBuilder = + new StringBuilder( + "RENAME COLUMN " + + BACK_QUOTE + + oldColumnName + + BACK_QUOTE + + SPACE + + "TO" + + SPACE + + BACK_QUOTE + + newColumnName + + BACK_QUOTE); + return sqlBuilder.toString(); + } + + private String updateColumnPositionFieldDefinition( + TableChange.UpdateColumnPosition updateColumnPosition, JdbcTable jdbcTable) { + if (updateColumnPosition.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = updateColumnPosition.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(jdbcTable, col); + StringBuilder columnDefinition = new StringBuilder(); + columnDefinition.append(MODIFY_COLUMN).append(quote(col)); + appendColumnDefinition(column, columnDefinition); + if (updateColumnPosition.getPosition() instanceof TableChange.First) { + columnDefinition.append("FIRST"); + } else if (updateColumnPosition.getPosition() instanceof TableChange.After) { + TableChange.After afterPosition = (TableChange.After) updateColumnPosition.getPosition(); + columnDefinition.append(AFTER).append(afterPosition.getColumn()); + } else { + Arrays.stream(jdbcTable.columns()) + .reduce((column1, column2) -> column2) + .map(Column::name) + .ifPresent(s -> columnDefinition.append(AFTER).append(s)); + } + return columnDefinition.toString(); + } + + private String deleteColumnFieldDefinition( + TableChange.DeleteColumn deleteColumn, JdbcTable jdbcTable) { + if (deleteColumn.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = deleteColumn.fieldName()[0]; + boolean colExists = true; + try { + getJdbcColumnFromTable(jdbcTable, col); + } catch (NoSuchColumnException noSuchColumnException) { + colExists = false; + } + if (!colExists) { + if (BooleanUtils.isTrue(deleteColumn.getIfExists())) { + return ""; + } else { + throw new IllegalArgumentException("Delete column does not exist: " + col); + } + } + return "DROP COLUMN " + BACK_QUOTE + col + BACK_QUOTE; + } + + private String updateColumnDefaultValueFieldDefinition( + TableChange.UpdateColumnDefaultValue updateColumnDefaultValue, JdbcTable jdbcTable) { + if (updateColumnDefaultValue.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = updateColumnDefaultValue.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(jdbcTable, col); + StringBuilder sqlBuilder = new StringBuilder(MODIFY_COLUMN + quote(col)); + JdbcColumn newColumn = + JdbcColumn.builder() + .withName(col) + .withType(column.dataType()) + .withNullable(column.nullable()) + .withComment(column.comment()) + .withDefaultValue(updateColumnDefaultValue.getNewDefaultValue()) + .build(); + return appendColumnDefinition(newColumn, sqlBuilder).toString(); + } + + private String updateColumnTypeFieldDefinition( + TableChange.UpdateColumnType updateColumnType, JdbcTable jdbcTable) { + if (updateColumnType.fieldName().length > 1) { + throw new UnsupportedOperationException(CLICKHOUSE_NOT_SUPPORT_NESTED_COLUMN_MSG); + } + String col = updateColumnType.fieldName()[0]; + JdbcColumn column = getJdbcColumnFromTable(jdbcTable, col); + StringBuilder sqlBuilder = new StringBuilder(MODIFY_COLUMN + quote(col)); + JdbcColumn newColumn = + JdbcColumn.builder() + .withName(col) + .withType(updateColumnType.getNewDataType()) + .withComment(column.comment()) + .withDefaultValue(DEFAULT_VALUE_NOT_SET) + .withNullable(column.nullable()) + .withAutoIncrement(column.autoIncrement()) + .build(); + return appendColumnDefinition(newColumn, sqlBuilder).toString(); + } + + private StringBuilder appendColumnDefinition(JdbcColumn column, StringBuilder sqlBuilder) { + // Add Nullable data type + String dataType = typeConverter.fromGravitino(column.dataType()); + if (column.nullable()) { + sqlBuilder.append(SPACE).append("Nullable(").append(dataType).append(")").append(SPACE); + } else { + sqlBuilder.append(SPACE).append(dataType).append(SPACE); + } + + // ck no support alter table with set nullable + + // Add DEFAULT value if specified + if (!DEFAULT_VALUE_NOT_SET.equals(column.defaultValue())) { + sqlBuilder + .append("DEFAULT ") + .append(columnDefaultValueConverter.fromGravitino(column.defaultValue())) + .append(SPACE); + } + + // Add column auto_increment if specified + // if (column.autoIncrement()) { + // sqlBuilder.append(CLICKHOUSE_AUTO_INCREMENT).append(" "); + // } + + // Add column comment if specified + if (StringUtils.isNotEmpty(column.comment())) { + sqlBuilder.append("COMMENT '").append(column.comment()).append("' "); + } + return sqlBuilder; + } + + private static String quote(String name) { + return BACK_QUOTE + name + BACK_QUOTE; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider b/catalogs/catalog-jdbc-clickhouse/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider new file mode 100644 index 00000000000..f0bd2f5f760 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider @@ -0,0 +1,19 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +org.apache.gravitino.catalog.clickhouse.ClickHouseCatalog \ No newline at end of file diff --git a/catalogs/catalog-jdbc-clickhouse/src/main/resources/jdbc-clickhouse.conf b/catalogs/catalog-jdbc-clickhouse/src/main/resources/jdbc-clickhouse.conf new file mode 100644 index 00000000000..3b0c336c4db --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/main/resources/jdbc-clickhouse.conf @@ -0,0 +1,23 @@ +# +# 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. +# +# jdbc-url = jdbc:clickhouse://localhost +# jdbc-user = strato +# jdbc-password = strato +# The selection of Driver should be filled in according to your needs, such as `com.clickhouse.jdbc.ClickHouseDriver` +# jdbc-driver = com.clickhouse.jdbc.ClickHouseDriver \ No newline at end of file diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseUtils.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseUtils.java new file mode 100644 index 00000000000..c4abaa64502 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/ClickHouseUtils.java @@ -0,0 +1,30 @@ +/* + * 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.gravitino.catalog.clickhouse.converter; + +import org.apache.gravitino.rel.expressions.NamedReference; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; +import org.apache.gravitino.rel.expressions.sorts.SortOrders; + +public class ClickHouseUtils { + + public static SortOrder[] getSortOrders(String colName) { + return new SortOrders.SortImpl[] {SortOrders.of(NamedReference.field(colName), null, null)}; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/TestClickHouseTypeConverter.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/TestClickHouseTypeConverter.java new file mode 100644 index 00000000000..a7945813a0a --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/converter/TestClickHouseTypeConverter.java @@ -0,0 +1,124 @@ +/* + * 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.gravitino.catalog.clickhouse.converter; + +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.BOOL; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.DATE; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.DATE32; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.DATETIME; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.DATETIME64; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.DECIMAL; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.FIXEDSTRING; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.FLOAT32; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.FLOAT64; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.INT16; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.INT32; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.INT64; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.INT8; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.STRING; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.UINT16; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.UINT32; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.UINT64; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.UINT8; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter.UUID; + +import org.apache.gravitino.catalog.jdbc.converter.JdbcTypeConverter; +import org.apache.gravitino.rel.types.Type; +import org.apache.gravitino.rel.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +/** Test class for {@link ClickHouseTypeConverter} */ +public class TestClickHouseTypeConverter { + + private static final ClickHouseTypeConverter CLICKHOUSE_TYPE_CONVERTER = + new ClickHouseTypeConverter(); + private static final String USER_DEFINED_TYPE = "user-defined"; + + @Test + public void testToGravitinoType() { + checkJdbcTypeToGravitinoType(Types.ByteType.get(), INT8, null, null); + checkJdbcTypeToGravitinoType(Types.ByteType.unsigned(), UINT8, null, null); + checkJdbcTypeToGravitinoType(Types.ShortType.get(), INT16, null, null); + checkJdbcTypeToGravitinoType(Types.ShortType.unsigned(), UINT16, null, null); + checkJdbcTypeToGravitinoType(Types.IntegerType.get(), INT32, null, null); + checkJdbcTypeToGravitinoType(Types.IntegerType.unsigned(), UINT32, null, null); + checkJdbcTypeToGravitinoType(Types.LongType.get(), INT64, null, null); + checkJdbcTypeToGravitinoType(Types.LongType.unsigned(), UINT64, null, null); + checkJdbcTypeToGravitinoType(Types.FloatType.get(), FLOAT32, null, null); + checkJdbcTypeToGravitinoType(Types.DoubleType.get(), FLOAT64, null, null); + checkJdbcTypeToGravitinoType(Types.DateType.get(), DATE, null, null); + checkJdbcTypeToGravitinoType(Types.DateType.get(), DATE32, null, null); + checkJdbcTypeToGravitinoType(Types.TimestampType.withoutTimeZone(), DATETIME, null, null); + checkJdbcTypeToGravitinoType(Types.TimestampType.withoutTimeZone(), DATETIME64, null, null); + checkJdbcTypeToGravitinoType(Types.DecimalType.of(10, 2), DECIMAL, 10, 2); + checkJdbcTypeToGravitinoType(Types.StringType.get(), STRING, 20, null); + checkJdbcTypeToGravitinoType(Types.FixedCharType.of(20), FIXEDSTRING, 20, null); + checkJdbcTypeToGravitinoType(Types.BooleanType.get(), BOOL, 20, null); + checkJdbcTypeToGravitinoType(Types.UUIDType.get(), UUID, 20, null); + checkJdbcTypeToGravitinoType( + Types.ExternalType.of(USER_DEFINED_TYPE), USER_DEFINED_TYPE, null, null); + } + + @Test + public void testFromGravitinoType() { + checkGravitinoTypeToJdbcType(INT8, Types.ByteType.get()); + checkGravitinoTypeToJdbcType(UINT8, Types.ByteType.unsigned()); + checkGravitinoTypeToJdbcType(INT16, Types.ShortType.get()); + checkGravitinoTypeToJdbcType(UINT16, Types.ShortType.unsigned()); + checkGravitinoTypeToJdbcType(INT32, Types.IntegerType.get()); + checkGravitinoTypeToJdbcType(UINT32, Types.IntegerType.unsigned()); + checkGravitinoTypeToJdbcType(INT64, Types.LongType.get()); + checkGravitinoTypeToJdbcType(UINT64, Types.LongType.unsigned()); + checkGravitinoTypeToJdbcType(FLOAT32, Types.FloatType.get()); + checkGravitinoTypeToJdbcType(FLOAT64, Types.DoubleType.get()); + checkGravitinoTypeToJdbcType(DATE, Types.DateType.get()); + checkGravitinoTypeToJdbcType(DATETIME, Types.TimestampType.withoutTimeZone()); + checkGravitinoTypeToJdbcType(DECIMAL + "(10,2)", Types.DecimalType.of(10, 2)); + checkGravitinoTypeToJdbcType(STRING, Types.VarCharType.of(20)); + checkGravitinoTypeToJdbcType(FIXEDSTRING + "(20)", Types.FixedCharType.of(20)); + checkGravitinoTypeToJdbcType(STRING, Types.StringType.get()); + checkGravitinoTypeToJdbcType(BOOL, Types.BooleanType.get()); + checkGravitinoTypeToJdbcType(UUID, Types.UUIDType.get()); + checkGravitinoTypeToJdbcType(USER_DEFINED_TYPE, Types.ExternalType.of(USER_DEFINED_TYPE)); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> CLICKHOUSE_TYPE_CONVERTER.fromGravitino(Types.UnparsedType.of(USER_DEFINED_TYPE))); + } + + protected void checkGravitinoTypeToJdbcType(String jdbcTypeName, Type gravitinoType) { + Assertions.assertEquals(jdbcTypeName, CLICKHOUSE_TYPE_CONVERTER.fromGravitino(gravitinoType)); + } + + protected void checkJdbcTypeToGravitinoType( + Type gravitinoType, String jdbcTypeName, Integer columnSize, Integer scale) { + JdbcTypeConverter.JdbcTypeBean typeBean = createTypeBean(jdbcTypeName, columnSize, scale); + Assertions.assertEquals(gravitinoType, CLICKHOUSE_TYPE_CONVERTER.toGravitino(typeBean)); + } + + protected static JdbcTypeConverter.JdbcTypeBean createTypeBean( + String typeName, Integer columnSize, Integer scale) { + return new JdbcTypeConverter.JdbcTypeBean(typeName) { + { + setColumnSize(columnSize); + setScale(scale); + } + }; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/AuditCatalogClickHouseIT.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/AuditCatalogClickHouseIT.java new file mode 100644 index 00000000000..158f18ad180 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/AuditCatalogClickHouseIT.java @@ -0,0 +1,188 @@ +/* + * 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.gravitino.catalog.clickhouse.integration.test; + +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseUtils.getSortOrders; +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_NOT_SET; + +import com.google.common.collect.Maps; +import java.io.IOException; +import java.sql.SQLException; +import java.util.Collections; +import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.Catalog; +import org.apache.gravitino.CatalogChange; +import org.apache.gravitino.Configs; +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.Schema; +import org.apache.gravitino.auth.AuthenticatorType; +import org.apache.gravitino.catalog.clickhouse.integration.test.service.ClickHouseService; +import org.apache.gravitino.catalog.jdbc.config.JdbcConfig; +import org.apache.gravitino.client.GravitinoMetalake; +import org.apache.gravitino.integration.test.container.ClickHouseContainer; +import org.apache.gravitino.integration.test.container.ContainerSuite; +import org.apache.gravitino.integration.test.util.BaseIT; +import org.apache.gravitino.integration.test.util.GravitinoITUtils; +import org.apache.gravitino.integration.test.util.TestDatabaseName; +import org.apache.gravitino.rel.Column; +import org.apache.gravitino.rel.Table; +import org.apache.gravitino.rel.TableChange; +import org.apache.gravitino.rel.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +@Tag("gravitino-docker-test") +public class AuditCatalogClickHouseIT extends BaseIT { + private static final ContainerSuite containerSuite = ContainerSuite.getInstance(); + public static final String metalakeName = + GravitinoITUtils.genRandomName("audit_clickhouse_metalake"); + private static final String expectUser = System.getProperty("user.name"); + protected static TestDatabaseName TEST_DB_NAME; + private static final String provider = "jdbc-clickhouse"; + + private static ClickHouseService clickhouseService; + private static ClickHouseContainer CLICKHOUSE_CONTAINER; + private static GravitinoMetalake metalake; + + @BeforeAll + public void startIntegrationTest() throws Exception { + Map configs = Maps.newHashMap(); + configs.put(Configs.AUTHENTICATORS.getKey(), AuthenticatorType.SIMPLE.name().toLowerCase()); + registerCustomConfigs(configs); + super.startIntegrationTest(); + + containerSuite.startClickHouseContainer( + TestDatabaseName.CLICKHOUSE_AUDIT_CATALOG_CLICKHOUSE_IT); + CLICKHOUSE_CONTAINER = containerSuite.getClickHouseContainer(); + TEST_DB_NAME = TestDatabaseName.CLICKHOUSE_AUDIT_CATALOG_CLICKHOUSE_IT; + clickhouseService = + new ClickHouseService(containerSuite.getClickHouseContainer(), TEST_DB_NAME); + createMetalake(); + } + + @AfterAll + public void stopIntegrationTest() throws IOException, InterruptedException { + client.dropMetalake(metalakeName, true); + clickhouseService.close(); + super.stopIntegrationTest(); + } + + @Test + public void testAuditCatalog() throws Exception { + String catalogName = GravitinoITUtils.genRandomName("audit_clickhouse_catalog"); + Catalog catalog = createCatalog(catalogName); + + Assertions.assertEquals(expectUser, catalog.auditInfo().creator()); + Assertions.assertEquals(catalog.auditInfo().creator(), catalog.auditInfo().lastModifier()); + Assertions.assertEquals( + catalog.auditInfo().createTime(), catalog.auditInfo().lastModifiedTime()); + catalog = metalake.alterCatalog(catalogName, CatalogChange.setProperty("key1", "value1")); + Assertions.assertEquals(expectUser, catalog.auditInfo().creator()); + Assertions.assertEquals(expectUser, catalog.auditInfo().lastModifier()); + + metalake.dropCatalog(catalogName, true); + } + + @Test + public void testAuditSchema() throws Exception { + String catalogName = GravitinoITUtils.genRandomName("audit_clickhouse_schema_catalog"); + String schemaName = GravitinoITUtils.genRandomName("audit_clickhouse_schema"); + Catalog catalog = createCatalog(catalogName); + Map prop = Maps.newHashMap(); + Schema schema = catalog.asSchemas().createSchema(schemaName, null, prop); + Assertions.assertEquals(expectUser, schema.auditInfo().creator()); + Assertions.assertNull(schema.auditInfo().lastModifier()); + + catalog.asSchemas().dropSchema(schemaName, true); + metalake.dropCatalog(catalogName, true); + } + + @Test + public void testAuditTable() throws Exception { + String catalogName = GravitinoITUtils.genRandomName("audit_clickhouse_table_catalog"); + String schemaName = GravitinoITUtils.genRandomName("audit_clickhouse_table_schema"); + String tableName = GravitinoITUtils.genRandomName("audit_clickhouse_table"); + Catalog catalog = createCatalog(catalogName); + Map properties = Maps.newHashMap(); + + Column col1 = + Column.of( + "col_1", Types.IntegerType.get(), "col_1_comment", false, false, DEFAULT_VALUE_NOT_SET); + + catalog.asSchemas().createSchema(schemaName, null, properties); + Table table = + catalog + .asTableCatalog() + .createTable( + NameIdentifier.of(schemaName, tableName), + new Column[] {col1}, + "comment", + properties, + null, + null, + getSortOrders("col_1")); + Assertions.assertEquals(expectUser, table.auditInfo().creator()); + Assertions.assertNull(table.auditInfo().lastModifier()); + table = + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.addColumn(new String[] {"col_4"}, Types.StringType.get())); + Assertions.assertEquals(expectUser, table.auditInfo().creator()); + Assertions.assertEquals(expectUser, table.auditInfo().lastModifier()); + + catalog.asTableCatalog().dropTable(NameIdentifier.of(schemaName, tableName)); + catalog.asSchemas().dropSchema(schemaName, true); + metalake.dropCatalog(catalogName, true); + } + + private static Catalog createCatalog(String catalogName) throws SQLException { + Map catalogProperties = Maps.newHashMap(); + + catalogProperties.put( + JdbcConfig.JDBC_URL.getKey(), + StringUtils.substring( + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME), + 0, + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME).lastIndexOf("/"))); + catalogProperties.put( + JdbcConfig.JDBC_DRIVER.getKey(), CLICKHOUSE_CONTAINER.getDriverClassName(TEST_DB_NAME)); + catalogProperties.put(JdbcConfig.USERNAME.getKey(), CLICKHOUSE_CONTAINER.getUsername()); + catalogProperties.put(JdbcConfig.PASSWORD.getKey(), CLICKHOUSE_CONTAINER.getPassword()); + + return metalake.createCatalog( + catalogName, Catalog.Type.RELATIONAL, provider, "comment", catalogProperties); + } + + private void createMetalake() { + GravitinoMetalake[] gravitinoMetalakes = client.listMetalakes(); + Assertions.assertEquals(0, gravitinoMetalakes.length); + + client.createMetalake(metalakeName, "comment", Collections.emptyMap()); + GravitinoMetalake loadMetalake = client.loadMetalake(metalakeName); + Assertions.assertEquals(metalakeName, loadMetalake.name()); + metalake = loadMetalake; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseDriverIT.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseDriverIT.java new file mode 100644 index 00000000000..327bf1d4872 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseDriverIT.java @@ -0,0 +1,29 @@ +/* + * 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.gravitino.catalog.clickhouse.integration.test; + +import org.junit.jupiter.api.Tag; + +@Tag("gravitino-docker-test") +public class CatalogClickHouseDriverIT extends CatalogClickHouseIT { + public CatalogClickHouseDriverIT() { + super(); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseIT.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseIT.java new file mode 100644 index 00000000000..2fc1aecae04 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/CatalogClickHouseIT.java @@ -0,0 +1,1652 @@ +/* + * 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.gravitino.catalog.clickhouse.integration.test; + +import static org.apache.gravitino.catalog.clickhouse.ClickHouseTablePropertiesMetadata.ENGINE.MERGETREE; +import static org.apache.gravitino.catalog.clickhouse.ClickHouseTablePropertiesMetadata.GRAVITINO_ENGINE_KEY; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseUtils.getSortOrders; +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_NOT_SET; +import static org.junit.jupiter.api.Assertions.assertThrows; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.sql.SQLException; +import java.time.LocalDate; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.Catalog; +import org.apache.gravitino.CatalogChange; +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.Namespace; +import org.apache.gravitino.Schema; +import org.apache.gravitino.SupportsSchemas; +import org.apache.gravitino.auth.AuthConstants; +import org.apache.gravitino.catalog.clickhouse.integration.test.service.ClickHouseService; +import org.apache.gravitino.catalog.jdbc.config.JdbcConfig; +import org.apache.gravitino.client.GravitinoMetalake; +import org.apache.gravitino.exceptions.NoSuchSchemaException; +import org.apache.gravitino.exceptions.NotFoundException; +import org.apache.gravitino.integration.test.container.ClickHouseContainer; +import org.apache.gravitino.integration.test.container.ContainerSuite; +import org.apache.gravitino.integration.test.util.BaseIT; +import org.apache.gravitino.integration.test.util.GravitinoITUtils; +import org.apache.gravitino.integration.test.util.ITUtils; +import org.apache.gravitino.integration.test.util.TestDatabaseName; +import org.apache.gravitino.rel.Column; +import org.apache.gravitino.rel.Table; +import org.apache.gravitino.rel.TableCatalog; +import org.apache.gravitino.rel.TableChange; +import org.apache.gravitino.rel.expressions.FunctionExpression; +import org.apache.gravitino.rel.expressions.UnparsedExpression; +import org.apache.gravitino.rel.expressions.distributions.Distribution; +import org.apache.gravitino.rel.expressions.distributions.Distributions; +import org.apache.gravitino.rel.expressions.literals.Literals; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; +import org.apache.gravitino.rel.expressions.transforms.Transform; +import org.apache.gravitino.rel.expressions.transforms.Transforms; +import org.apache.gravitino.rel.indexes.Index; +import org.apache.gravitino.rel.indexes.Indexes; +import org.apache.gravitino.rel.types.Decimal; +import org.apache.gravitino.rel.types.Types; +import org.apache.gravitino.utils.RandomNameUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; +import org.junit.jupiter.api.condition.EnabledIf; + +@Tag("gravitino-docker-test") +@TestInstance(Lifecycle.PER_CLASS) +public class CatalogClickHouseIT extends BaseIT { + private static final ContainerSuite containerSuite = ContainerSuite.getInstance(); + private static final String provider = "jdbc-clickhouse"; + + public String metalakeName = GravitinoITUtils.genRandomName("clickhouse_it_metalake"); + public String catalogName = GravitinoITUtils.genRandomName("clickhouse_it_catalog"); + public String schemaName = GravitinoITUtils.genRandomName("clickhouse_it_schema"); + public String tableName = GravitinoITUtils.genRandomName("clickhouse_it_table"); + public String alertTableName = "alert_table_name"; + public String table_comment = "table_comment"; + + // ClickHouse doesn't support schema comment + public String schema_comment = null; + public String CLICKHOUSE_COL_NAME1 = "clickhouse_col_name1"; + public String CLICKHOUSE_COL_NAME2 = "clickhouse_col_name2"; + public String CLICKHOUSE_COL_NAME3 = "clickhouse_col_name3"; + public String CLICKHOUSE_COL_NAME4 = "clickhouse_col_name4"; + public String CLICKHOUSE_COL_NAME5 = "clickhouse_col_name5"; + + private GravitinoMetalake metalake; + + protected Catalog catalog; + + private ClickHouseService clickhouseService; + + private ClickHouseContainer CLICKHOUSE_CONTAINER; + + private TestDatabaseName TEST_DB_NAME; + + public static final String defaultClickhouseImageName = "clickhouse:8.0"; + + protected String clickhouseImageName = defaultClickhouseImageName; + + boolean SupportColumnDefaultValueExpression() { + return true; + } + + @BeforeAll + public void startup() throws IOException, SQLException { + TEST_DB_NAME = TestDatabaseName.CLICKHOUSE_CATALOG_CLICKHOUSE_IT; + + // if (clickhouseImageName.equals("clickhouse:5.7")) { + // + // containerSuite.startClickHouseVersion5Container(TestDatabaseName.CLICKHOUSE_CATALOG_CLICKHOUSE_IT); + // CLICKHOUSE_CONTAINER = containerSuite.getClickHouseVersion5Container(); + // } else { + containerSuite.startClickHouseContainer(TEST_DB_NAME); + CLICKHOUSE_CONTAINER = containerSuite.getClickHouseContainer(); + // } + + clickhouseService = new ClickHouseService(CLICKHOUSE_CONTAINER, TEST_DB_NAME); + createMetalake(); + createCatalog(); + createSchema(); + } + + @AfterAll + public void stop() { + clearTableAndSchema(); + metalake.disableCatalog(catalogName); + metalake.dropCatalog(catalogName); + client.disableMetalake(metalakeName); + client.dropMetalake(metalakeName); + clickhouseService.close(); + } + + @AfterEach + public void resetSchema() { + clearTableAndSchema(); + createSchema(); + } + + private void clearTableAndSchema() { + NameIdentifier[] nameIdentifiers = + catalog.asTableCatalog().listTables(Namespace.of(schemaName)); + for (NameIdentifier nameIdentifier : nameIdentifiers) { + catalog.asTableCatalog().dropTable(nameIdentifier); + } + catalog.asSchemas().dropSchema(schemaName, false); + } + + private void createMetalake() { + GravitinoMetalake[] gravitinoMetalakes = client.listMetalakes(); + Assertions.assertEquals(0, gravitinoMetalakes.length); + + client.createMetalake(metalakeName, "comment", Collections.emptyMap()); + GravitinoMetalake loadMetalake = client.loadMetalake(metalakeName); + Assertions.assertEquals(metalakeName, loadMetalake.name()); + + metalake = loadMetalake; + } + + private void createCatalog() throws SQLException { + Map catalogProperties = Maps.newHashMap(); + + catalogProperties.put( + JdbcConfig.JDBC_URL.getKey(), + StringUtils.substring( + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME), + 0, + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME).lastIndexOf("/"))); + catalogProperties.put( + JdbcConfig.JDBC_DRIVER.getKey(), CLICKHOUSE_CONTAINER.getDriverClassName(TEST_DB_NAME)); + catalogProperties.put(JdbcConfig.USERNAME.getKey(), CLICKHOUSE_CONTAINER.getUsername()); + catalogProperties.put(JdbcConfig.PASSWORD.getKey(), CLICKHOUSE_CONTAINER.getPassword()); + + Catalog createdCatalog = + metalake.createCatalog( + catalogName, Catalog.Type.RELATIONAL, provider, "comment", catalogProperties); + Catalog loadCatalog = metalake.loadCatalog(catalogName); + Assertions.assertEquals(createdCatalog, loadCatalog); + + catalog = loadCatalog; + } + + private void createSchema() { + Map prop = Maps.newHashMap(); + Schema createdSchema = null; + try { + createdSchema = catalog.asSchemas().createSchema(schemaName, schema_comment, prop); + } catch (Exception ex) { + System.out.println(ex); + } + + Schema loadSchema = catalog.asSchemas().loadSchema(schemaName); + Assertions.assertEquals(createdSchema.name(), loadSchema.name()); + prop.forEach((key, value) -> Assertions.assertEquals(loadSchema.properties().get(key), value)); + } + + private Column[] createColumns() { + Column col1 = Column.of(CLICKHOUSE_COL_NAME1, Types.IntegerType.get(), "col_1_comment"); + Column col2 = Column.of(CLICKHOUSE_COL_NAME2, Types.DateType.get(), "col_2_comment"); + Column col3 = + Column.of( + CLICKHOUSE_COL_NAME3, + Types.StringType.get(), + "col_3_comment", + false, + false, + DEFAULT_VALUE_NOT_SET); + + return new Column[] {col1, col2, col3}; + } + + private Column[] createColumnsWithDefaultValue() { + return new Column[] { + Column.of( + CLICKHOUSE_COL_NAME1, + Types.FloatType.get(), + "col_1_comment", + false, + false, + Literals.of("1.23", Types.FloatType.get())), + Column.of( + CLICKHOUSE_COL_NAME2, + Types.TimestampType.withoutTimeZone(), + "col_2_comment", + false, + false, + FunctionExpression.of("now")), + Column.of( + CLICKHOUSE_COL_NAME3, + Types.VarCharType.of(255), + "col_3_comment", + true, + false, + Literals.NULL), + Column.of( + CLICKHOUSE_COL_NAME4, + Types.IntegerType.get(), + "col_4_comment", + false, + false, + Literals.of("1000", Types.IntegerType.get())), + Column.of( + CLICKHOUSE_COL_NAME5, + Types.DecimalType.of(3, 2), + "col_5_comment", + true, + false, + Literals.of("1.23", Types.DecimalType.of(3, 2))) + }; + } + + private Map createProperties() { + Map properties = Maps.newHashMap(); + properties.put(GRAVITINO_ENGINE_KEY, MERGETREE.getValue()); + return properties; + } + + @Test + void testOperationClickhouseSchema() { + SupportsSchemas schemas = catalog.asSchemas(); + Namespace namespace = Namespace.of(metalakeName, catalogName); + // list schema check. + String[] nameIdentifiers = schemas.listSchemas(); + Set schemaNames = Sets.newHashSet(nameIdentifiers); + Assertions.assertTrue(schemaNames.contains(schemaName)); + + NameIdentifier[] clickhouseNamespaces = clickhouseService.listSchemas(namespace); + schemaNames = + Arrays.stream(clickhouseNamespaces).map(NameIdentifier::name).collect(Collectors.toSet()); + Assertions.assertTrue(schemaNames.contains(schemaName)); + + // create schema check. + String testSchemaName = GravitinoITUtils.genRandomName("test_schema_1"); + NameIdentifier schemaIdent = NameIdentifier.of(metalakeName, catalogName, testSchemaName); + schemas.createSchema(testSchemaName, schema_comment, Collections.emptyMap()); + nameIdentifiers = schemas.listSchemas(); + schemaNames = Sets.newHashSet(nameIdentifiers); + Assertions.assertTrue(schemaNames.contains(testSchemaName)); + + clickhouseNamespaces = clickhouseService.listSchemas(namespace); + schemaNames = + Arrays.stream(clickhouseNamespaces).map(NameIdentifier::name).collect(Collectors.toSet()); + Assertions.assertTrue(schemaNames.contains(testSchemaName)); + + Map emptyMap = Collections.emptyMap(); + Assertions.assertThrows( + RuntimeException.class, + () -> { + schemas.createSchema(testSchemaName, schema_comment, emptyMap); + }); + + // drop schema check. + schemas.dropSchema(testSchemaName, false); + Assertions.assertThrows(NoSuchSchemaException.class, () -> schemas.loadSchema(testSchemaName)); + Assertions.assertThrows( + NoSuchSchemaException.class, () -> clickhouseService.loadSchema(schemaIdent)); + + nameIdentifiers = schemas.listSchemas(); + schemaNames = Sets.newHashSet(nameIdentifiers); + Assertions.assertFalse(schemaNames.contains(testSchemaName)); + Assertions.assertFalse(schemas.dropSchema("no-exits", false)); + TableCatalog tableCatalog = catalog.asTableCatalog(); + + // create failed check. + NameIdentifier table = NameIdentifier.of(testSchemaName, "test_table"); + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> + tableCatalog.createTable( + table, + createColumns(), + table_comment, + createProperties(), + null, + Distributions.NONE, + getSortOrders(CLICKHOUSE_COL_NAME3))); + // drop schema failed check. + Assertions.assertFalse(schemas.dropSchema(schemaIdent.name(), true)); + Assertions.assertFalse(schemas.dropSchema(schemaIdent.name(), false)); + Assertions.assertFalse(() -> tableCatalog.dropTable(table)); + clickhouseNamespaces = clickhouseService.listSchemas(Namespace.empty()); + schemaNames = + Arrays.stream(clickhouseNamespaces).map(NameIdentifier::name).collect(Collectors.toSet()); + Assertions.assertTrue(schemaNames.contains(schemaName)); + } + + @Test + void testCreateAndLoadClickhouseTable() { + // Create table from Gravitino API + Column[] columns = createColumns(); + + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, tableName); + Distribution distribution = Distributions.NONE; + + Transform[] partitioning = Transforms.EMPTY_TRANSFORM; + + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + partitioning, + distribution, + getSortOrders(CLICKHOUSE_COL_NAME3)); + + Table loadTable = tableCatalog.loadTable(tableIdentifier); + Assertions.assertEquals(tableName, loadTable.name()); + Assertions.assertEquals(table_comment, loadTable.comment()); + Map resultProp = loadTable.properties(); + for (Map.Entry entry : properties.entrySet()) { + Assertions.assertTrue(resultProp.containsKey(entry.getKey())); + Assertions.assertEquals(entry.getValue(), resultProp.get(entry.getKey())); + } + Assertions.assertEquals(loadTable.columns().length, columns.length); + for (int i = 0; i < columns.length; i++) { + ITUtils.assertColumn(columns[i], loadTable.columns()[i]); + } + } + + @Test + void testColumnNameWithKeyWords() { + // Create table from Gravitino API + Column[] columns = { + Column.of("integer", Types.IntegerType.get(), "integer", false, false, DEFAULT_VALUE_NOT_SET), + Column.of("long", Types.LongType.get(), "long"), + Column.of("float", Types.FloatType.get(), "float"), + Column.of("double", Types.DoubleType.get(), "double"), + Column.of("decimal", Types.DecimalType.of(10, 3), "decimal"), + Column.of("date", Types.DateType.get(), "date"), + Column.of("time", Types.TimeType.get(), "time") + }; + + String name = GravitinoITUtils.genRandomName("table") + "_keyword"; + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, name); + Distribution distribution = Distributions.NONE; + + Transform[] partitioning = Transforms.EMPTY_TRANSFORM; + + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + Table createdTable = + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + partitioning, + distribution, + getSortOrders("integer")); + Assertions.assertEquals(createdTable.name(), name); + } + + @Test + // ClickHouse support column default value expression after 8.0.13 + // see https://dev.clickhouse.com/doc/refman/8.0/en/data-type-defaults.html + @EnabledIf("SupportColumnDefaultValueExpression") + void testColumnDefaultValue() { + Column col1 = + Column.of( + CLICKHOUSE_COL_NAME1, + Types.IntegerType.get(), + "col_1_comment", + false, + false, + FunctionExpression.of("rand")); + Column col2 = + Column.of( + CLICKHOUSE_COL_NAME2, + Types.TimestampType.withoutTimeZone(), + "col_2_comment", + false, + false, + FunctionExpression.of("now")); + Column col3 = + Column.of( + CLICKHOUSE_COL_NAME3, + Types.VarCharType.of(255), + "col_3_comment", + true, + false, + Literals.NULL); + Column col4 = + Column.of( + CLICKHOUSE_COL_NAME4, Types.StringType.get(), "col_4_comment", false, false, null); + Column col5 = + Column.of( + CLICKHOUSE_COL_NAME5, + Types.VarCharType.of(255), + "col_5_comment", + true, + false, + Literals.stringLiteral("now()")); + + Column[] newColumns = new Column[] {col1, col2, col3, col4, col5}; + + NameIdentifier tableIdent = + NameIdentifier.of(schemaName, GravitinoITUtils.genRandomName("clickhouse_it_table")); + catalog + .asTableCatalog() + .createTable( + tableIdent, + newColumns, + null, + ImmutableMap.of(), + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(CLICKHOUSE_COL_NAME1)); + Table createdTable = catalog.asTableCatalog().loadTable(tableIdent); + Assertions.assertEquals( + UnparsedExpression.of("rand()"), createdTable.columns()[0].defaultValue()); + Assertions.assertEquals( + UnparsedExpression.of("now()"), createdTable.columns()[1].defaultValue()); + Assertions.assertEquals(Literals.NULL, createdTable.columns()[2].defaultValue()); + Assertions.assertEquals(DEFAULT_VALUE_NOT_SET, createdTable.columns()[3].defaultValue()); + Assertions.assertEquals( + Literals.stringLiteral("now()"), createdTable.columns()[4].defaultValue()); + } + + @Test + // ClickHouse support column default value expression after 8.0.13 + // see https://dev.clickhouse.com/doc/refman/8.0/en/data-type-defaults.html + @EnabledIf("SupportColumnDefaultValueExpression") + void testColumnDefaultValueConverter() { + // test convert from ClickHouse to Gravitino + String tableName = GravitinoITUtils.genRandomName("test_default_value"); + String fullTableName = schemaName + "." + tableName; + String sql = + "CREATE TABLE " + + fullTableName + + " (\n" + + " int_col_1 int default 0x01AF,\n" + + " int_col_2 int default (rand()),\n" + + " int_col_3 int default 3,\n" + + " unsigned_int_col_1 INT UNSIGNED default 1,\n" + + " unsigned_bigint_col_1 BIGINT(20) UNSIGNED default 0,\n" + + " double_col_1 double default 123.45,\n" + + " varchar20_col_1 varchar(20) default '10',\n" + + " varchar100_col_1 varchar(100) default 'now()',\n" + + " varchar200_col_1 varchar(200) default 'curdate()',\n" + + " varchar200_col_2 varchar(200) default (today()),\n" + + " varchar200_col_3 varchar(200) default (now()),\n" + + " datetime_col_1 datetime default now(),\n" + + " datetime_col_2 datetime default now(),\n" + + " datetime_col_3 datetime default null,\n" + + " datetime_col_4 datetime default 19830905,\n" + + " date_col_1 date default (today()),\n" + + " date_col_2 date,\n" + + " date_col_3 date DEFAULT (today() + INTERVAL 1 YEAR),\n" + + " date_col_4 date DEFAULT (today()),\n" + + " date_col_5 date DEFAULT '2024-04-01',\n" + + " timestamp_col_1 timestamp default '2012-12-31 11:30:45',\n" + + " timestamp_col_2 timestamp default 19830905,\n" + + " timestamp_col_3 timestamp(6) default now(),\n" + + " decimal_6_2_col_1 decimal(6, 2) default 1.2,\n" + + " bit_col_1 bit default '1'\n" + + ") order by int_col_1;\n"; + + clickhouseService.executeQuery(sql); + Table loadedTable = + catalog.asTableCatalog().loadTable(NameIdentifier.of(schemaName, tableName)); + + for (Column column : loadedTable.columns()) { + // try { + switch (column.name()) { + case "int_col_1": + Assertions.assertEquals(Literals.integerLiteral(431), column.defaultValue()); + break; + case "int_col_2": + Assertions.assertEquals(UnparsedExpression.of("rand()"), column.defaultValue()); + break; + case "int_col_3": + Assertions.assertEquals(Literals.integerLiteral(3), column.defaultValue()); + break; + case "unsigned_int_col_1": + Assertions.assertEquals(Literals.unsignedIntegerLiteral(1L), column.defaultValue()); + break; + case "unsigned_bigint_col_1": + Assertions.assertEquals( + Literals.unsignedLongLiteral(Decimal.of("0")), column.defaultValue()); + break; + case "double_col_1": + Assertions.assertEquals(Literals.doubleLiteral(123.45), column.defaultValue()); + break; + case "varchar20_col_1": + Assertions.assertEquals(Literals.stringLiteral("10"), column.defaultValue()); + break; + case "varchar100_col_1": + Assertions.assertEquals(Literals.stringLiteral("now()"), column.defaultValue()); + break; + case "varchar200_col_1": + Assertions.assertEquals(Literals.stringLiteral("curdate()"), column.defaultValue()); + break; + case "varchar200_col_2": + Assertions.assertEquals(Literals.stringLiteral("today()"), column.defaultValue()); + break; + case "varchar200_col_3": + Assertions.assertEquals(Literals.stringLiteral("now()"), column.defaultValue()); + break; + case "datetime_col_1": + case "datetime_col_2": + Assertions.assertEquals(UnparsedExpression.of("now()"), column.defaultValue()); + break; + case "datetime_col_3": + Assertions.assertEquals(Literals.NULL, column.defaultValue()); + break; + case "datetime_col_4": + Assertions.assertEquals(UnparsedExpression.of("19830905"), column.defaultValue()); + break; + case "date_col_1": + Assertions.assertEquals(UnparsedExpression.of("today()"), column.defaultValue()); + break; + case "date_col_2": + Assertions.assertEquals(DEFAULT_VALUE_NOT_SET, column.defaultValue()); + break; + case "date_col_3": + Assertions.assertEquals( + UnparsedExpression.of("today() + toIntervalYear(1)"), column.defaultValue()); + break; + case "date_col_4": + Assertions.assertEquals(UnparsedExpression.of("today()"), column.defaultValue()); + break; + case "date_col_5": + Assertions.assertEquals( + Literals.dateLiteral(LocalDate.of(2024, 4, 1)), column.defaultValue()); + break; + case "timestamp_col_1": + Assertions.assertEquals( + Literals.timestampLiteral("2012-12-31T11:30:45"), column.defaultValue()); + break; + case "timestamp_col_2": + Assertions.assertEquals(UnparsedExpression.of("19830905"), column.defaultValue()); + break; + case "timestamp_col_3": + Assertions.assertEquals(UnparsedExpression.of("now()"), column.defaultValue()); + break; + case "decimal_6_2_col_1": + Assertions.assertEquals( + Literals.decimalLiteral(Decimal.of("1.2", 6, 2)), column.defaultValue()); + break; + case "bit_col_1": + Assertions.assertEquals( + Literals.unsignedLongLiteral(Decimal.of("1")), column.defaultValue()); + break; + default: + Assertions.fail( + "Unexpected column name: " + + column.name() + + ", default value: " + + column.defaultValue()); + } + // } catch (Throwable ex) { + // System.err.println(ex); + // for (Object obj : ex.getStackTrace()) { + // System.err.println(obj); + // } + // } + } + } + + @Test + void testColumnTypeConverter() { + // test convert from ClickHouse to Gravitino + String tableName = GravitinoITUtils.genRandomName("test_type_converter"); + String fullTableName = schemaName + "." + tableName; + String sql = + "CREATE TABLE " + + fullTableName + + " (\n" + + " tinyint_col tinyint,\n" + + " smallint_col smallint,\n" + + " int_col int,\n" + + " bigint_col bigint,\n" + + " float_col float,\n" + + " double_col double,\n" + + " date_col date,\n" + + " time_col time,\n" + + " timestamp_col timestamp,\n" + + " datetime_col datetime,\n" + + " decimal_6_2_col decimal(6, 2),\n" + + " varchar20_col varchar(20),\n" + + " text_col text,\n" + // + " binary_col binary,\n" + + " blob_col blob\n" + + ") order by tinyint_col;\n"; + + clickhouseService.executeQuery(sql); + Table loadedTable = + catalog.asTableCatalog().loadTable(NameIdentifier.of(schemaName, tableName)); + + for (Column column : loadedTable.columns()) { + switch (column.name()) { + case "tinyint_col": + Assertions.assertEquals(Types.ByteType.get(), column.dataType()); + break; + case "smallint_col": + Assertions.assertEquals(Types.ShortType.get(), column.dataType()); + break; + case "int_col": + Assertions.assertEquals(Types.IntegerType.get(), column.dataType()); + break; + case "bigint_col": + Assertions.assertEquals(Types.LongType.get(), column.dataType()); + break; + case "float_col": + Assertions.assertEquals(Types.FloatType.get(), column.dataType()); + break; + case "double_col": + Assertions.assertEquals(Types.DoubleType.get(), column.dataType()); + break; + case "date_col": + Assertions.assertEquals(Types.DateType.get(), column.dataType()); + break; + case "time_col": + Assertions.assertEquals(Types.LongType.get(), column.dataType()); + break; + case "timestamp_col": + Assertions.assertEquals(Types.TimestampType.withoutTimeZone(), column.dataType()); + break; + case "datetime_col": + Assertions.assertEquals(Types.TimestampType.withoutTimeZone(), column.dataType()); + break; + case "decimal_6_2_col": + Assertions.assertEquals(Types.DecimalType.of(6, 2), column.dataType()); + break; + case "varchar20_col": + Assertions.assertEquals(Types.StringType.get(), column.dataType()); + break; + case "text_col": + Assertions.assertEquals(Types.StringType.get(), column.dataType()); + break; + case "binary_col": + Assertions.assertEquals(Types.BinaryType.get(), column.dataType()); + break; + case "blob_col": + Assertions.assertEquals(Types.StringType.get(), column.dataType()); + break; + default: + Assertions.fail("Unexpected column name: " + column.name()); + } + } + } + + @Test + void testAlterAndDropClickhouseTable() { + Column[] columns = createColumns(); + catalog + .asTableCatalog() + .createTable( + NameIdentifier.of(schemaName, tableName), + columns, + table_comment, + createProperties(), + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(CLICKHOUSE_COL_NAME3)); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.rename(alertTableName), + TableChange.updateComment(table_comment + "_new")); + }); + + catalog + .asTableCatalog() + .alterTable(NameIdentifier.of(schemaName, tableName), TableChange.rename(alertTableName)); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, alertTableName), + TableChange.updateComment(table_comment + "_new")); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, alertTableName), + TableChange.addColumn(new String[] {"col_4"}, Types.StringType.get())); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, alertTableName), + TableChange.renameColumn(new String[] {CLICKHOUSE_COL_NAME2}, "col_2_new")); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, alertTableName), + TableChange.updateColumnType( + new String[] {CLICKHOUSE_COL_NAME1}, Types.IntegerType.get())); + + Table table = catalog.asTableCatalog().loadTable(NameIdentifier.of(schemaName, alertTableName)); + Assertions.assertEquals(alertTableName, table.name()); + + Assertions.assertEquals(CLICKHOUSE_COL_NAME1, table.columns()[0].name()); + Assertions.assertEquals(Types.IntegerType.get(), table.columns()[0].dataType()); + + Assertions.assertEquals("col_2_new", table.columns()[1].name()); + Assertions.assertEquals(Types.DateType.get(), table.columns()[1].dataType()); + Assertions.assertEquals("col_2_comment", table.columns()[1].comment()); + + Assertions.assertEquals(CLICKHOUSE_COL_NAME3, table.columns()[2].name()); + Assertions.assertEquals(Types.StringType.get(), table.columns()[2].dataType()); + Assertions.assertEquals("col_3_comment", table.columns()[2].comment()); + + Assertions.assertEquals("col_4", table.columns()[3].name()); + Assertions.assertEquals(Types.StringType.get(), table.columns()[3].dataType()); + Assertions.assertNull(table.columns()[3].comment()); + Assertions.assertNotNull(table.auditInfo()); + Assertions.assertNotNull(table.auditInfo().createTime()); + Assertions.assertNotNull(table.auditInfo().creator()); + Assertions.assertNotNull(table.auditInfo().lastModifiedTime()); + Assertions.assertNotNull(table.auditInfo().lastModifier()); + + Column col1 = + Column.of("name", Types.StringType.get(), "comment", false, false, DEFAULT_VALUE_NOT_SET); + Column col2 = Column.of("address", Types.StringType.get(), "comment"); + Column col3 = Column.of("date_of_birth", Types.DateType.get(), "comment"); + + Column[] newColumns = new Column[] {col1, col2, col3}; + NameIdentifier tableIdentifier = + NameIdentifier.of(schemaName, GravitinoITUtils.genRandomName("CatalogJdbcIT_table")); + catalog + .asTableCatalog() + .createTable( + tableIdentifier, + newColumns, + table_comment, + ImmutableMap.of(), + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("name")); + + TableCatalog tableCatalog = catalog.asTableCatalog(); + TableChange change = + TableChange.updateColumnPosition( + new String[] {"no_column"}, TableChange.ColumnPosition.first()); + NotFoundException notFoundException = + assertThrows( + NotFoundException.class, () -> tableCatalog.alterTable(tableIdentifier, change)); + Assertions.assertTrue(notFoundException.getMessage().contains("no_column")); + + catalog + .asTableCatalog() + .alterTable( + tableIdentifier, + TableChange.updateColumnPosition( + new String[] {col1.name()}, TableChange.ColumnPosition.after(col2.name()))); + + Table updateColumnPositionTable = catalog.asTableCatalog().loadTable(tableIdentifier); + + Column[] updateCols = updateColumnPositionTable.columns(); + Assertions.assertEquals(3, updateCols.length); + Assertions.assertEquals(col2.name(), updateCols[0].name()); + Assertions.assertEquals(col1.name(), updateCols[1].name()); + Assertions.assertEquals(col3.name(), updateCols[2].name()); + + Assertions.assertDoesNotThrow( + () -> + catalog + .asTableCatalog() + .alterTable( + tableIdentifier, + TableChange.deleteColumn(new String[] {col3.name()}, true), + TableChange.deleteColumn(new String[] {col2.name()}, true))); + Table delColTable = catalog.asTableCatalog().loadTable(tableIdentifier); + Assertions.assertEquals(1, delColTable.columns().length); + Assertions.assertEquals(col1.name(), delColTable.columns()[0].name()); + + Assertions.assertDoesNotThrow( + () -> { + catalog.asTableCatalog().dropTable(tableIdentifier); + }); + } + + @Test + void testUpdateColumnDefaultValue() { + Column[] columns = createColumnsWithDefaultValue(); + Table table = + catalog + .asTableCatalog() + .createTable( + NameIdentifier.of(schemaName, tableName), + columns, + null, + ImmutableMap.of(), + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(CLICKHOUSE_COL_NAME1)); + + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, table.auditInfo().creator()); + Assertions.assertNull(table.auditInfo().lastModifier()); + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[0].name()}, Literals.of("1.2345", Types.FloatType.get()))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[1].name()}, FunctionExpression.of("now"))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[2].name()}, Literals.of("hello", Types.VarCharType.of(255)))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[3].name()}, Literals.of("2000", Types.IntegerType.get()))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[4].name()}, Literals.of("2.34", Types.DecimalType.of(3, 2)))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[0].name()}, Literals.of("1.2345", Types.FloatType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[1].name()}, FunctionExpression.of("now")), + TableChange.updateColumnDefaultValue( + new String[] {columns[2].name()}, Literals.of("hello", Types.VarCharType.of(255))), + TableChange.updateColumnDefaultValue( + new String[] {columns[3].name()}, Literals.of("2000", Types.IntegerType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[4].name()}, Literals.of("2.34", Types.DecimalType.of(3, 2)))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[0].name()}, Literals.of("1.2345", Types.FloatType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[1].name()}, FunctionExpression.of("now")), + TableChange.updateColumnDefaultValue( + new String[] {columns[2].name()}, Literals.of("hello", Types.VarCharType.of(255))), + TableChange.updateColumnDefaultValue( + new String[] {columns[3].name()}, Literals.of("2000", Types.IntegerType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[4].name()}, Literals.of("2.34", Types.DecimalType.of(3, 2)))); + + catalog + .asTableCatalog() + .alterTable( + NameIdentifier.of(schemaName, tableName), + TableChange.updateColumnDefaultValue( + new String[] {columns[0].name()}, Literals.of("1.2345", Types.FloatType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[1].name()}, FunctionExpression.of("now")), + TableChange.updateColumnDefaultValue( + new String[] {columns[2].name()}, Literals.of("hello", Types.VarCharType.of(255))), + TableChange.updateColumnDefaultValue( + new String[] {columns[3].name()}, Literals.of("2000", Types.IntegerType.get())), + TableChange.updateColumnDefaultValue( + new String[] {columns[4].name()}, Literals.of("2.34", Types.DecimalType.of(3, 2)))); + + table = catalog.asTableCatalog().loadTable(NameIdentifier.of(schemaName, tableName)); + + Assertions.assertEquals( + Literals.of("1.2345", Types.FloatType.get()), table.columns()[0].defaultValue()); + Assertions.assertEquals(UnparsedExpression.of("now()"), table.columns()[1].defaultValue()); + Assertions.assertEquals( + Literals.of("hello", Types.StringType.get()), table.columns()[2].defaultValue()); + Assertions.assertEquals( + Literals.of("2000", Types.IntegerType.get()), table.columns()[3].defaultValue()); + Assertions.assertEquals( + Literals.of("2.34", Types.DecimalType.of(3, 2)), table.columns()[4].defaultValue()); + } + + @Test + void testDropClickHouseDatabase() { + String schemaName = GravitinoITUtils.genRandomName("clickhouse_schema").toLowerCase(); + String tableName = GravitinoITUtils.genRandomName("clickhouse_table").toLowerCase(); + + catalog + .asSchemas() + .createSchema(schemaName, null, ImmutableMap.builder().build()); + + catalog + .asTableCatalog() + .createTable( + NameIdentifier.of(schemaName, tableName), + createColumns(), + "Created by Gravitino client", + ImmutableMap.builder().build(), + getSortOrders(CLICKHOUSE_COL_NAME3)); + + // Try to drop a database, and cascade equals to false, it should not be + // allowed. + Throwable excep = + Assertions.assertThrows( + RuntimeException.class, () -> catalog.asSchemas().dropSchema(schemaName, false)); + Assertions.assertTrue(excep.getMessage().contains("the value of cascade should be true.")); + // Assertions.assertTrue( + // excep.getMessage().contains("has sub-entities, you should remove sub-entities + // first")); + + // Check the database still exists + catalog.asSchemas().loadSchema(schemaName); + + // Try to drop a database, and cascade equals to true, it should be allowed. + catalog.asSchemas().dropSchema(schemaName, true); + // Check database has been dropped + SupportsSchemas schemas = catalog.asSchemas(); + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> { + schemas.loadSchema(schemaName); + }); + } + + @Test + public void testSchemaComment() { + final String testSchemaName = "test"; + RuntimeException exception = + Assertions.assertThrowsExactly( + UnsupportedOperationException.class, + () -> catalog.asSchemas().createSchema(testSchemaName, "comment", null)); + Assertions.assertTrue( + exception.getMessage().contains("Doesn't support setting schema comment: comment")); + + // test null comment + String testSchemaName2 = "test2"; + Schema schema = catalog.asSchemas().createSchema(testSchemaName2, "", null); + Assertions.assertTrue(StringUtils.isEmpty(schema.comment())); + schema = catalog.asSchemas().loadSchema(testSchemaName2); + Assertions.assertTrue(StringUtils.isEmpty(schema.comment())); + catalog.asSchemas().dropSchema(testSchemaName2, true); + } + + @Test + public void testBackQuoteTable() { + Column col1 = Column.of("create", Types.LongType.get(), "id", false, false, null); + Column col2 = Column.of("delete", Types.ByteType.get(), "yes", false, false, null); + Column col3 = Column.of("show", Types.DateType.get(), "comment", false, false, null); + Column col4 = Column.of("status", Types.VarCharType.of(255), "code", false, false, null); + Column[] newColumns = new Column[] {col1, col2, col3, col4}; + TableCatalog tableCatalog = catalog.asTableCatalog(); + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, "table"); + Assertions.assertDoesNotThrow( + () -> + tableCatalog.createTable( + tableIdentifier, + newColumns, + table_comment, + Collections.emptyMap(), + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("create"), + Indexes.EMPTY_INDEXES)); + + Assertions.assertDoesNotThrow(() -> tableCatalog.loadTable(tableIdentifier)); + + Assertions.assertDoesNotThrow( + () -> + tableCatalog.alterTable( + tableIdentifier, + TableChange.addColumn( + new String[] {"int"}, + Types.StringType.get(), + TableChange.ColumnPosition.after("status")))); + + Table table = tableCatalog.loadTable(tableIdentifier); + for (Column column : table.columns()) { + System.out.println(column.name()); + } + + Assertions.assertDoesNotThrow( + () -> + tableCatalog.alterTable( + tableIdentifier, TableChange.renameColumn(new String[] {"int"}, "varchar"))); + + Assertions.assertDoesNotThrow( + () -> + tableCatalog.alterTable( + tableIdentifier, TableChange.deleteColumn(new String[] {"varchar"}, true))); + + Assertions.assertDoesNotThrow(() -> tableCatalog.dropTable(tableIdentifier)); + } + + @Test + void testClickHouseSpecialTableName() { + // Test create many indexes with name success. + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + + String t1_name = "t112"; + Column t1_col = Column.of(t1_name, Types.LongType.get(), "id", false, false, null); + Column[] columns = {t1_col}; + + Index[] t1_indexes = {Indexes.primary("u1_key", new String[][] {{t1_name}})}; + + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, t1_name); + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("t112"), + t1_indexes); + + String t2_name = "t212"; + Column t2_col = Column.of(t2_name, Types.LongType.get(), "id", false, false, null); + Index[] t2_indexes = {Indexes.primary("u2_key", new String[][] {{t2_name}})}; + columns = new Column[] {t2_col}; + tableIdentifier = NameIdentifier.of(schemaName, t2_name); + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(t2_name), + t2_indexes); + + String t3_name = "t_12"; + Column t3_col = Column.of(t3_name, Types.LongType.get(), "id", false, false, null); + Index[] t3_indexes = {Indexes.primary("u3_key", new String[][] {{t3_name}})}; + columns = new Column[] {t3_col}; + tableIdentifier = NameIdentifier.of(schemaName, t3_name); + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(t3_name), + t3_indexes); + + String t4_name = "_1__"; + Column t4_col = Column.of(t4_name, Types.LongType.get(), "id", false, false, null); + Index[] t4_indexes = {Indexes.primary("u4_key", new String[][] {{t4_name}})}; + columns = new Column[] {t4_col}; + tableIdentifier = NameIdentifier.of(schemaName, t4_name); + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders(t4_name), + t4_indexes); + + Table t1 = tableCatalog.loadTable(NameIdentifier.of(schemaName, t1_name)); + Arrays.stream(t1.columns()).anyMatch(c -> Objects.equals(c.name(), "t112")); + ITUtils.assertionsTableInfo( + t1_name, + table_comment, + Arrays.asList(t1_col), + properties, + null, + Transforms.EMPTY_TRANSFORM, + t1); + + Table t2 = tableCatalog.loadTable(NameIdentifier.of(schemaName, t2_name)); + Arrays.stream(t2.columns()).anyMatch(c -> Objects.equals(c.name(), "t212")); + ITUtils.assertionsTableInfo( + t2_name, + table_comment, + Arrays.asList(t2_col), + properties, + t2_indexes, + Transforms.EMPTY_TRANSFORM, + t2); + + Table t3 = tableCatalog.loadTable(NameIdentifier.of(schemaName, t3_name)); + Arrays.stream(t3.columns()).anyMatch(c -> Objects.equals(c.name(), "t_12")); + ITUtils.assertionsTableInfo( + t3_name, + table_comment, + Arrays.asList(t3_col), + properties, + t3_indexes, + Transforms.EMPTY_TRANSFORM, + t3); + + Table t4 = tableCatalog.loadTable(NameIdentifier.of(schemaName, t4_name)); + Arrays.stream(t4.columns()).anyMatch(c -> Objects.equals(c.name(), "_1__")); + ITUtils.assertionsTableInfo( + t4_name, + table_comment, + Arrays.asList(t4_col), + properties, + t4_indexes, + Transforms.EMPTY_TRANSFORM, + t4); + } + + @Test + void testClickHouseIllegalTableName() { + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + String table_name = "t123"; + + String t1_name = table_name + "`; DROP TABLE important_table; -- "; + Column t1_col = Column.of(t1_name, Types.LongType.get(), "id", false, false, null); + Column[] columns = {t1_col}; + Index[] t1_indexes = {Indexes.unique("u1_key", new String[][] {{t1_name}})}; + NameIdentifier tableIdentifier = + NameIdentifier.of(metalakeName, catalogName, schemaName, t1_name); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + tableCatalog.createTable( + tableIdentifier, + columns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + new SortOrder[0], + t1_indexes); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asTableCatalog().dropTable(tableIdentifier); + }); + + String t2_name = table_name + "`; SLEEP(10); -- "; + Column t2_col = Column.of(t2_name, Types.LongType.get(), "id", false, false, null); + Index[] t2_indexes = {Indexes.unique("u2_key", new String[][] {{t2_name}})}; + Column[] columns2 = new Column[] {t2_col}; + NameIdentifier tableIdentifier2 = + NameIdentifier.of(metalakeName, catalogName, schemaName, t2_name); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + tableCatalog.createTable( + tableIdentifier2, + columns2, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + new SortOrder[0], + t2_indexes); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asTableCatalog().dropTable(tableIdentifier2); + }); + + String t3_name = + table_name + "`; UPDATE Users SET password = 'newpassword' WHERE username = 'admin'; -- "; + Column t3_col = Column.of(t3_name, Types.LongType.get(), "id", false, false, null); + Index[] t3_indexes = {Indexes.unique("u3_key", new String[][] {{t3_name}})}; + Column[] columns3 = new Column[] {t3_col}; + NameIdentifier tableIdentifier3 = + NameIdentifier.of(metalakeName, catalogName, schemaName, t3_name); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + tableCatalog.createTable( + tableIdentifier3, + columns3, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + new SortOrder[0], + t3_indexes); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asTableCatalog().dropTable(tableIdentifier3); + }); + + String invalidInput = StringUtils.repeat("a", 65); + Column t4_col = Column.of(invalidInput, Types.LongType.get(), "id", false, false, null); + Index[] t4_indexes = {Indexes.unique("u4_key", new String[][] {{invalidInput}})}; + Column[] columns4 = new Column[] {t4_col}; + NameIdentifier tableIdentifier4 = + NameIdentifier.of(metalakeName, catalogName, schemaName, invalidInput); + + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + tableCatalog.createTable( + tableIdentifier4, + columns4, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + new SortOrder[0], + t4_indexes); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asTableCatalog().dropTable(tableIdentifier4); + }); + } + + @Test + void testClickHouseTableNameCaseSensitive() { + Column col1 = Column.of("col_1", Types.LongType.get(), "id", false, false, null); + Column col2 = Column.of("col_2", Types.ByteType.get(), "yes", false, false, null); + Column col3 = Column.of("col_3", Types.DateType.get(), "comment", false, false, null); + Column col4 = Column.of("col_4", Types.StringType.get(), "code", false, false, null); + Column col5 = Column.of("col_5", Types.StringType.get(), "config", false, false, null); + Column[] newColumns = new Column[] {col1, col2, col3, col4, col5}; + + Index[] indexes = new Index[0]; + + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, "tableName"); + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + Table createdTable = + tableCatalog.createTable( + tableIdentifier, + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + indexes); + ITUtils.assertionsTableInfo( + "tableName", + table_comment, + Arrays.asList(newColumns), + properties, + indexes, + Transforms.EMPTY_TRANSFORM, + createdTable); + Table table = tableCatalog.loadTable(tableIdentifier); + ITUtils.assertionsTableInfo( + "tableName", + table_comment, + Arrays.asList(newColumns), + properties, + indexes, + Transforms.EMPTY_TRANSFORM, + table); + + // Test create table with same name but different case + NameIdentifier tableIdentifier2 = NameIdentifier.of(schemaName, "TABLENAME"); + + Table tableAgain = + Assertions.assertDoesNotThrow( + () -> + tableCatalog.createTable( + tableIdentifier2, + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + indexes)); + Assertions.assertEquals("TABLENAME", tableAgain.name()); + + table = tableCatalog.loadTable(tableIdentifier2); + ITUtils.assertionsTableInfo( + "TABLENAME", + table_comment, + Arrays.asList(newColumns), + properties, + indexes, + Transforms.EMPTY_TRANSFORM, + table); + } + + @Test + void testNameSpec() { + // test operate illegal schema name from ClickHouse + String testSchemaName = "//"; + String sql = String.format("CREATE DATABASE `%s`", testSchemaName); + clickhouseService.executeQuery(sql); + + Schema schema = catalog.asSchemas().loadSchema(testSchemaName); + Assertions.assertEquals(testSchemaName, schema.name()); + + String[] schemaIdents = catalog.asSchemas().listSchemas(); + Assertions.assertTrue(Arrays.stream(schemaIdents).anyMatch(s -> s.equals(testSchemaName))); + + Assertions.assertTrue(catalog.asSchemas().dropSchema(testSchemaName, false)); + Assertions.assertFalse(catalog.asSchemas().schemaExists(testSchemaName)); + + // test operate illegal table name from ClickHouse + clickhouseService.executeQuery(sql); + String testTableName = "//"; + sql = + String.format("CREATE TABLE `%s`.`%s` (id int) order by id", testSchemaName, testTableName); + clickhouseService.executeQuery(sql); + NameIdentifier tableIdent = NameIdentifier.of(testSchemaName, testTableName); + + Table table = catalog.asTableCatalog().loadTable(tableIdent); + Assertions.assertEquals(testTableName, table.name()); + + NameIdentifier[] tableIdents = + catalog.asTableCatalog().listTables(Namespace.of(testSchemaName)); + Assertions.assertTrue(Arrays.stream(tableIdents).anyMatch(t -> t.name().equals(testTableName))); + + Assertions.assertTrue(catalog.asTableCatalog().dropTable(tableIdent)); + Assertions.assertFalse(catalog.asTableCatalog().tableExists(tableIdent)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> { + catalog.asTableCatalog().purgeTable(tableIdent); + }); + catalog.asSchemas().dropSchema(testSchemaName, true); + + // sql injection + String schemaName = RandomNameUtils.genRandomName("ct_db"); + Map properties = new HashMap<>(); + String comment = null; + + // should throw an exception with string that might contain SQL injection + String sqlInjection = schemaName + "`; DROP TABLE important_table; -- "; + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().createSchema(sqlInjection, comment, properties); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().dropSchema(sqlInjection, false); + }); + + String sqlInjection1 = schemaName + "`; SLEEP(10); -- "; + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().createSchema(sqlInjection1, comment, properties); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().dropSchema(sqlInjection1, false); + }); + + String sqlInjection2 = + schemaName + "`; UPDATE Users SET password = 'newpassword' WHERE username = 'admin'; -- "; + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().createSchema(sqlInjection2, comment, properties); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().dropSchema(sqlInjection2, false); + }); + + // should throw an exception with input that has more than 64 characters + String invalidInput = StringUtils.repeat("a", 65); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().createSchema(invalidInput, comment, properties); + }); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + catalog.asSchemas().dropSchema(invalidInput, false); + }); + } + + @Test + void testClickHouseSchemaNameCaseSensitive() { + Column col1 = Column.of("col_1", Types.LongType.get(), "id", false, false, null); + Column col2 = Column.of("col_2", Types.VarCharType.of(255), "code", false, false, null); + Column col3 = Column.of("col_3", Types.VarCharType.of(255), "config", false, false, null); + Column[] newColumns = new Column[] {col1, col2, col3}; + + // Index[] indexes = new Index[] {Indexes.unique("u1_key", new String[][] {{"col_2"}, + // {"col_3"}})}; + + String[] schemas = {"db_", "db_1", "db_2", "db12"}; + SupportsSchemas schemaSupport = catalog.asSchemas(); + + for (String schema : schemas) { + schemaSupport.createSchema(schema, null, Collections.emptyMap()); + Assertions.assertNotNull(schemaSupport.loadSchema(schema)); + } + + Set schemaNames = Sets.newHashSet(schemaSupport.listSchemas()); + + Assertions.assertTrue(schemaNames.containsAll(Arrays.asList(schemas))); + + String tableName = "test1"; + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + + for (String schema : schemas) { + tableCatalog.createTable( + NameIdentifier.of(schema, tableName), + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + null); + tableCatalog.createTable( + NameIdentifier.of(schema, GravitinoITUtils.genRandomName("test2")), + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + Indexes.EMPTY_INDEXES); + } + + for (String schema : schemas) { + NameIdentifier[] nameIdentifiers = tableCatalog.listTables(Namespace.of(schema)); + Assertions.assertEquals(2, nameIdentifiers.length); + Assertions.assertTrue( + Arrays.stream(nameIdentifiers) + .map(NameIdentifier::name) + .collect(Collectors.toSet()) + .stream() + .anyMatch(n -> n.equals(tableName))); + } + + for (String schema : schemas) { + schemaSupport.dropSchema(schema, true); + } + } + + @Test + void testUnparsedTypeConverter() { + String tableName = GravitinoITUtils.genRandomName("test_unparsed_type"); + clickhouseService.executeQuery( + String.format( + "CREATE TABLE %s.%s (bit_col IPv4) order by bit_col ;", schemaName, tableName)); + Table loadedTable = + catalog.asTableCatalog().loadTable(NameIdentifier.of(schemaName, tableName)); + Assertions.assertEquals(Types.ExternalType.of("IPv4"), loadedTable.columns()[0].dataType()); + } + + @Test + void testAddColumnDefaultValue() { + Column col1 = Column.of("col_1", Types.LongType.get(), "uid", false, false, null); + Column col2 = Column.of("col_2", Types.ByteType.get(), "yes", true, false, null); + Column col3 = Column.of("col_3", Types.StringType.get(), "comment", true, false, null); + String tableName = "default_value_table"; + Column[] newColumns = new Column[] {col1, col2, col3}; + + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, tableName); + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + tableCatalog.createTable( + tableIdentifier, + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + Indexes.EMPTY_INDEXES); + + Column col4 = + Column.of("col_4", Types.LongType.get(), "col4", true, false, DEFAULT_VALUE_NOT_SET); + tableCatalog.alterTable( + tableIdentifier, + TableChange.addColumn( + new String[] {col4.name()}, + col4.dataType(), + col4.comment(), + TableChange.ColumnPosition.defaultPos())); + + Table table = tableCatalog.loadTable(tableIdentifier); + newColumns = new Column[] {col1, col2, col3, col4}; + + // Assertions.assertEquals(col4.defaultValue(), table.columns()[3].defaultValue()); + + ITUtils.assertionsTableInfo( + tableName, + table_comment, + Arrays.asList(newColumns), + properties, + Indexes.EMPTY_INDEXES, + Transforms.EMPTY_TRANSFORM, + table); + } + + @Test + public void testClickHouseIntegerTypes() { + Column col1 = Column.of("col_1", Types.ByteType.get(), "byte type", false, false, null); + Column col2 = + Column.of("col_2", Types.ByteType.unsigned(), "byte unsigned type", true, false, null); + Column col3 = Column.of("col_3", Types.ShortType.get(), "short type", true, false, null); + Column col4 = + Column.of("col_4", Types.ShortType.unsigned(), "short unsigned type ", true, false, null); + Column col5 = Column.of("col_5", Types.IntegerType.get(), "integer type", true, false, null); + Column col6 = + Column.of( + "col_6", Types.IntegerType.unsigned(), "integer unsigned type", true, false, null); + Column col7 = Column.of("col_7", Types.LongType.get(), "long type", true, false, null); + Column col8 = + Column.of("col_8", Types.LongType.unsigned(), "long unsigned type", true, false, null); + String tableName = "default_integer_types_table"; + Column[] newColumns = new Column[] {col1, col2, col3, col4, col5, col6, col7, col8}; + + NameIdentifier tableIdentifier = NameIdentifier.of(schemaName, tableName); + Map properties = createProperties(); + TableCatalog tableCatalog = catalog.asTableCatalog(); + tableCatalog.createTable( + tableIdentifier, + newColumns, + table_comment, + properties, + Transforms.EMPTY_TRANSFORM, + Distributions.NONE, + getSortOrders("col_1"), + Indexes.EMPTY_INDEXES); + + Table table = tableCatalog.loadTable(tableIdentifier); + Assertions.assertEquals(8, table.columns().length); + Column[] columns = table.columns(); + Assertions.assertEquals(columns[0].dataType().simpleString(), "byte"); + Assertions.assertEquals(columns[1].dataType().simpleString(), "byte unsigned"); + Assertions.assertEquals(columns[2].dataType().simpleString(), "short"); + Assertions.assertEquals(columns[3].dataType().simpleString(), "short unsigned"); + Assertions.assertEquals(columns[4].dataType().simpleString(), "integer"); + Assertions.assertEquals(columns[5].dataType().simpleString(), "integer unsigned"); + Assertions.assertEquals(columns[6].dataType().simpleString(), "long"); + Assertions.assertEquals(columns[7].dataType().simpleString(), "long unsigned"); + } + + @Test + void testAlterCatalogProperties() throws SQLException { + Map catalogProperties = Maps.newHashMap(); + String testCatalogName = GravitinoITUtils.genRandomName("clickhouse_it_catalog"); + + catalogProperties.put( + JdbcConfig.JDBC_URL.getKey(), + StringUtils.substring( + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME), + 0, + CLICKHOUSE_CONTAINER.getJdbcUrl(TEST_DB_NAME).lastIndexOf("/"))); + catalogProperties.put( + JdbcConfig.JDBC_DRIVER.getKey(), CLICKHOUSE_CONTAINER.getDriverClassName(TEST_DB_NAME)); + catalogProperties.put(JdbcConfig.USERNAME.getKey(), CLICKHOUSE_CONTAINER.getUsername()); + + String password = CLICKHOUSE_CONTAINER.getPassword(); + String wrongPassword = password + "wrong"; + catalogProperties.put(JdbcConfig.PASSWORD.getKey(), wrongPassword); + + metalake.createCatalog( + testCatalogName, Catalog.Type.RELATIONAL, provider, "comment", catalogProperties); + Catalog loadCatalog = metalake.loadCatalog(testCatalogName); + + Assertions.assertThrows( + Exception.class, () -> loadCatalog.asSchemas().createSchema("test", "", null)); + metalake.alterCatalog( + testCatalogName, CatalogChange.setProperty(JdbcConfig.PASSWORD.getKey(), password)); + + Assertions.assertDoesNotThrow(() -> loadCatalog.asSchemas().createSchema("test", "", null)); + + loadCatalog.asSchemas().dropSchema("test", true); + metalake.dropCatalog(testCatalogName, true); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/service/ClickHouseService.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/service/ClickHouseService.java new file mode 100644 index 00000000000..d71d9208017 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/integration/test/service/ClickHouseService.java @@ -0,0 +1,111 @@ +/* + * 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.gravitino.catalog.clickhouse.integration.test.service; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.NameIdentifier; +import org.apache.gravitino.Namespace; +import org.apache.gravitino.catalog.jdbc.JdbcSchema; +import org.apache.gravitino.exceptions.NoSuchSchemaException; +import org.apache.gravitino.integration.test.container.ClickHouseContainer; +import org.apache.gravitino.integration.test.util.TestDatabaseName; +import org.apache.gravitino.meta.AuditInfo; + +public class ClickHouseService { + + private Connection connection; + + public ClickHouseService(ClickHouseContainer clickhouseContainer, TestDatabaseName testDBName) { + String username = clickhouseContainer.getUsername(); + String password = clickhouseContainer.getPassword(); + + try { + connection = + DriverManager.getConnection( + StringUtils.substring( + clickhouseContainer.getJdbcUrl(testDBName), + 0, + clickhouseContainer.getJdbcUrl(testDBName).lastIndexOf("/")), + username, + password); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public NameIdentifier[] listSchemas(Namespace namespace) { + List databases = new ArrayList<>(); + try (Statement statement = connection.createStatement(); + ResultSet resultSet = statement.executeQuery("SHOW DATABASES")) { + while (resultSet.next()) { + databases.add(resultSet.getString(1)); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return databases.stream() + .map(s -> NameIdentifier.of(ArrayUtils.add(namespace.levels(), s))) + .toArray(NameIdentifier[]::new); + } + + public JdbcSchema loadSchema(NameIdentifier schemaIdent) { + String databaseName = schemaIdent.name(); + String query = "SELECT * FROM information_schema.SCHEMATA WHERE SCHEMA_NAME = ?"; + try (PreparedStatement preparedStatement = connection.prepareStatement(query)) { + preparedStatement.setString(1, databaseName); + + // Execute the query + try (ResultSet resultSet = preparedStatement.executeQuery()) { + if (!resultSet.next()) { + throw new NoSuchSchemaException( + "Database %s could not be found in information_schema.SCHEMATA", databaseName); + } + String schemaName = resultSet.getString("SCHEMA_NAME"); + return JdbcSchema.builder().withName(schemaName).withAuditInfo(AuditInfo.EMPTY).build(); + } + } catch (final SQLException se) { + throw new RuntimeException(se); + } + } + + public void executeQuery(String sql) { + try (Statement statement = connection.createStatement()) { + statement.execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + public void close() { + try { + connection.close(); + } catch (SQLException e) { + // ignore + } + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouse.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouse.java new file mode 100644 index 00000000000..3591ce16adf --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouse.java @@ -0,0 +1,76 @@ +/* + * 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.gravitino.catalog.clickhouse.operation; + +import com.google.common.collect.Maps; +import java.sql.SQLException; +import java.util.Collections; +import java.util.Map; +import javax.sql.DataSource; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseColumnDefaultValueConverter; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseExceptionConverter; +import org.apache.gravitino.catalog.clickhouse.converter.ClickHouseTypeConverter; +import org.apache.gravitino.catalog.jdbc.TestJdbc; +import org.apache.gravitino.catalog.jdbc.config.JdbcConfig; +import org.apache.gravitino.catalog.jdbc.utils.DataSourceUtils; +import org.apache.gravitino.integration.test.container.ClickHouseContainer; +import org.apache.gravitino.integration.test.container.ContainerSuite; +import org.apache.gravitino.integration.test.util.TestDatabaseName; +import org.junit.jupiter.api.BeforeAll; + +public class TestClickHouse extends TestJdbc { + public static final String defaultClickhouseImageName = "clickhouse:8.0"; + protected static final ContainerSuite containerSuite = ContainerSuite.getInstance(); + protected static TestDatabaseName TEST_DB_NAME; + + @BeforeAll + public static void startup() throws Exception { + ContainerSuite containerSuite = ContainerSuite.getInstance(); + TEST_DB_NAME = TestDatabaseName.CLICKHOUSE_CLICKHOUSE_ABSTRACT_IT; + containerSuite.startClickHouseContainer(TEST_DB_NAME); + DataSource dataSource = DataSourceUtils.createDataSource(getClickHouseCatalogProperties()); + + DATABASE_OPERATIONS = new ClickHouseDatabaseOperations(); + TABLE_OPERATIONS = new ClickHouseTableOperations(); + JDBC_EXCEPTION_CONVERTER = new ClickHouseExceptionConverter(); + DATABASE_OPERATIONS.initialize(dataSource, JDBC_EXCEPTION_CONVERTER, Collections.emptyMap()); + TABLE_OPERATIONS.initialize( + dataSource, + JDBC_EXCEPTION_CONVERTER, + new ClickHouseTypeConverter(), + new ClickHouseColumnDefaultValueConverter(), + Collections.emptyMap()); + } + + protected static Map getClickHouseCatalogProperties() throws SQLException { + Map catalogProperties = Maps.newHashMap(); + + ClickHouseContainer clickHouseContainer = containerSuite.getClickHouseContainer(); + + String jdbcUrl = clickHouseContainer.getJdbcUrl(TEST_DB_NAME); + + catalogProperties.put(JdbcConfig.JDBC_URL.getKey(), jdbcUrl); + catalogProperties.put( + JdbcConfig.JDBC_DRIVER.getKey(), clickHouseContainer.getDriverClassName(TEST_DB_NAME)); + catalogProperties.put(JdbcConfig.USERNAME.getKey(), clickHouseContainer.getUsername()); + catalogProperties.put(JdbcConfig.PASSWORD.getKey(), clickHouseContainer.getPassword()); + + return catalogProperties; + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseCatalogOperations.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseCatalogOperations.java new file mode 100644 index 00000000000..3c4fd2a8ff2 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseCatalogOperations.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.catalog.clickhouse.operation; + +import java.sql.SQLException; +import org.apache.gravitino.catalog.clickhouse.ClickHouseCatalog; +import org.apache.gravitino.catalog.jdbc.JdbcCatalogOperations; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +@Tag("gravitino-docker-test") +public class TestClickHouseCatalogOperations extends TestClickHouse { + + @Test + public void testCheckJDBCDriver() throws SQLException { + JdbcCatalogOperations catalogOperations = + new JdbcCatalogOperations(null, null, DATABASE_OPERATIONS, TABLE_OPERATIONS, null); + catalogOperations.initialize(getClickHouseCatalogProperties(), null, new ClickHouseCatalog()); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseDatabaseOperations.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseDatabaseOperations.java new file mode 100644 index 00000000000..5245e969963 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseDatabaseOperations.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.catalog.clickhouse.operation; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.gravitino.catalog.jdbc.JdbcColumn; +import org.apache.gravitino.rel.expressions.NamedReference; +import org.apache.gravitino.rel.expressions.distributions.Distributions; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; +import org.apache.gravitino.rel.expressions.sorts.SortOrders; +import org.apache.gravitino.rel.expressions.transforms.Transform; +import org.apache.gravitino.rel.indexes.Index; +import org.apache.gravitino.rel.types.Types; +import org.apache.gravitino.utils.RandomNameUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +@Tag("gravitino-docker-test") +public class TestClickHouseDatabaseOperations extends TestClickHouse { + + @Test + public void testBaseOperationDatabase() { + String databaseName = RandomNameUtils.genRandomName("ct_db"); + Map properties = new HashMap<>(); + // Clickhouse database creation does not support incoming comments. + String comment = null; + List databases = DATABASE_OPERATIONS.listDatabases(); + ((ClickHouseDatabaseOperations) DATABASE_OPERATIONS) + .createSysDatabaseNameSet() + .forEach( + sysClickhouseDatabaseName -> + Assertions.assertFalse(databases.contains(sysClickhouseDatabaseName))); + testBaseOperation(databaseName, properties, comment); + testDropDatabase(databaseName); + } + + @Test + void testDropTableWithSpecificName() { + String databaseName = RandomNameUtils.genRandomName("ct_db") + "-abc-" + "end"; + Map properties = new HashMap<>(); + DATABASE_OPERATIONS.create(databaseName, null, properties); + DATABASE_OPERATIONS.delete(databaseName, false); + + databaseName = RandomNameUtils.genRandomName("ct_db") + "_end"; + DATABASE_OPERATIONS.create(databaseName, null, properties); + + String tableName = RandomStringUtils.randomAlphabetic(16) + "_op_table"; + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + columns.add( + JdbcColumn.builder() + .withName("col_1") + .withType(Types.VarCharType.of(100)) + .withComment("test_comment") + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_2") + .withType(Types.VarCharType.of(100)) + .withComment("test_comment2") + .withNullable(true) + .build()); + + Index[] indexes = new Index[] {}; + + SortOrder[] sortOrders = + new SortOrders.SortImpl[] {SortOrders.of(NamedReference.field("col_1"), null, null)}; + + TABLE_OPERATIONS.create( + databaseName, + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + properties, + new Transform[0], + Distributions.NONE, + indexes, + sortOrders); + DATABASE_OPERATIONS.delete(databaseName, true); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseTableOperations.java b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseTableOperations.java new file mode 100644 index 00000000000..3242fbfdfec --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/java/org/apache/gravitino/catalog/clickhouse/operation/TestClickHouseTableOperations.java @@ -0,0 +1,717 @@ +/* + * 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.gravitino.catalog.clickhouse.operation; + +import static org.apache.gravitino.catalog.clickhouse.ClickHouseTablePropertiesMetadata.CLICKHOUSE_ENGINE_KEY; +import static org.apache.gravitino.catalog.clickhouse.converter.ClickHouseUtils.getSortOrders; +import static org.apache.gravitino.rel.Column.DEFAULT_VALUE_NOT_SET; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.catalog.jdbc.JdbcColumn; +import org.apache.gravitino.catalog.jdbc.JdbcTable; +import org.apache.gravitino.exceptions.GravitinoRuntimeException; +import org.apache.gravitino.rel.TableChange; +import org.apache.gravitino.rel.expressions.distributions.Distributions; +import org.apache.gravitino.rel.expressions.literals.Literals; +import org.apache.gravitino.rel.expressions.transforms.Transforms; +import org.apache.gravitino.rel.indexes.Index; +import org.apache.gravitino.rel.indexes.Indexes; +import org.apache.gravitino.rel.types.Decimal; +import org.apache.gravitino.rel.types.Type; +import org.apache.gravitino.rel.types.Types; +import org.apache.gravitino.utils.RandomNameUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +@Tag("gravitino-docker-test") +public class TestClickHouseTableOperations extends TestClickHouse { + private static final Type STRING = Types.StringType.get(); + private static final Type INT = Types.IntegerType.get(); + private static final Type LONG = Types.LongType.get(); + + @Test + public void testOperationTable() { + String tableName = RandomStringUtils.randomAlphabetic(16) + "_op_table"; + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + columns.add( + JdbcColumn.builder() + .withName("col_1") + .withType(STRING) + .withComment("test_comment") + .withNullable(true) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_2") + .withType(INT) + .withNullable(false) + .withComment("set primary key") + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_3") + .withType(INT) + .withNullable(true) + .withDefaultValue(Literals.NULL) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_4") + .withType(STRING) + .withDefaultValue(Literals.of("hello world", STRING)) + .withNullable(false) + .build()); + Map properties = new HashMap<>(); + + Index[] indexes = new Index[] {}; + // create table + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + properties, + null, + Distributions.NONE, + indexes, + getSortOrders("col_2")); + + // list table + List tables = TABLE_OPERATIONS.listTables(TEST_DB_NAME.toString()); + Assertions.assertTrue(tables.contains(tableName)); + + // load table + JdbcTable load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + assertionsTableInfo( + tableName, tableComment, columns, properties, indexes, Transforms.EMPTY_TRANSFORM, load); + + // rename table + String newName = "new_table"; + Assertions.assertDoesNotThrow( + () -> TABLE_OPERATIONS.rename(TEST_DB_NAME.toString(), tableName, newName)); + Assertions.assertDoesNotThrow(() -> TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), newName)); + + // alter table + JdbcColumn newColumn = + JdbcColumn.builder() + .withName("col_5") + .withType(STRING) + .withComment("new_add") + .withNullable(false) // + // .withDefaultValue(Literals.of("hello test", STRING)) + .build(); + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + newName, + TableChange.addColumn( + new String[] {newColumn.name()}, + newColumn.dataType(), + newColumn.comment(), + TableChange.ColumnPosition.after("col_1"), + newColumn.nullable(), + newColumn.autoIncrement(), + newColumn.defaultValue()) + // , + // TableChange.setProperty(CLICKHOUSE_ENGINE_KEY, "InnoDB")); + // properties.put(CLICKHOUSE_ENGINE_KEY, "InnoDB" + ); + load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), newName); + List alterColumns = + new ArrayList() { + { + add(columns.get(0)); + add(newColumn); + add(columns.get(1)); + add(columns.get(2)); + add(columns.get(3)); + } + }; + assertionsTableInfo( + newName, tableComment, alterColumns, properties, indexes, Transforms.EMPTY_TRANSFORM, load); + + // Detect unsupported properties + TableChange setProperty = TableChange.setProperty(CLICKHOUSE_ENGINE_KEY, "ABC"); + UnsupportedOperationException gravitinoRuntimeException = + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> TABLE_OPERATIONS.alterTable(TEST_DB_NAME.toString(), newName, setProperty)); + Assertions.assertTrue( + StringUtils.contains( + gravitinoRuntimeException.getMessage(), + "alter table properties in ck is not supported")); + + // delete column + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + newName, + TableChange.deleteColumn(new String[] {newColumn.name()}, true)); + load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), newName); + assertionsTableInfo( + newName, tableComment, columns, properties, indexes, Transforms.EMPTY_TRANSFORM, load); + + TableChange deleteColumn = TableChange.deleteColumn(new String[] {newColumn.name()}, false); + IllegalArgumentException illegalArgumentException = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> TABLE_OPERATIONS.alterTable(TEST_DB_NAME.toString(), newName, deleteColumn)); + Assertions.assertEquals( + "Delete column does not exist: " + newColumn.name(), illegalArgumentException.getMessage()); + Assertions.assertDoesNotThrow( + () -> + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + newName, + TableChange.deleteColumn(new String[] {newColumn.name()}, true))); + + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + newName, + TableChange.deleteColumn(new String[] {newColumn.name()}, true)); + Assertions.assertTrue( + TABLE_OPERATIONS.drop(TEST_DB_NAME.toString(), newName), "table should be dropped"); + + GravitinoRuntimeException exception = + Assertions.assertThrows( + GravitinoRuntimeException.class, + () -> TABLE_OPERATIONS.drop(TEST_DB_NAME.toString(), newName)); + Assertions.assertTrue(StringUtils.contains(exception.getMessage(), "does not exist")); + } + + @Test + public void testAlterTable() { + String tableName = RandomStringUtils.randomAlphabetic(16) + "_al_table"; + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + JdbcColumn col_1 = + JdbcColumn.builder() + .withName("col_1") + .withType(INT) + .withComment("id") + .withNullable(false) + .withDefaultValue(Literals.integerLiteral(0)) + .build(); + columns.add(col_1); + JdbcColumn col_2 = + JdbcColumn.builder() + .withName("col_2") + .withType(STRING) + .withComment("name") + .withDefaultValue(Literals.of("hello world", STRING)) + .withNullable(false) + .build(); + columns.add(col_2); + JdbcColumn col_3 = + JdbcColumn.builder() + .withName("col_3") + .withType(STRING) + .withComment("name") + .withDefaultValue(Literals.NULL) + .build(); + // `col_1` int NOT NULL COMMENT 'id' , + // `col_2` STRING(255) NOT NULL DEFAULT 'hello world' COMMENT 'name' , + // `col_3` STRING(255) NULL DEFAULT NULL COMMENT 'name' , + columns.add(col_3); + Map properties = new HashMap<>(); + + // create table + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + properties, + null, + Distributions.NONE, + null, + getSortOrders("col_2")); + JdbcTable load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + assertionsTableInfo( + tableName, tableComment, columns, properties, null, Transforms.EMPTY_TRANSFORM, load); + + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + tableName, + TableChange.updateColumnType(new String[] {col_1.name()}, LONG)); + + load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + + // After modifying the type, some attributes of the corresponding column are not + // supported. + columns.clear(); + col_1 = + JdbcColumn.builder() + .withName(col_1.name()) + .withType(LONG) + .withComment(col_1.comment()) + .withNullable(col_1.nullable()) + .withDefaultValue(Literals.longLiteral(0L)) + .build(); + columns.add(col_1); + columns.add(col_2); + columns.add(col_3); + assertionsTableInfo( + tableName, tableComment, columns, properties, null, Transforms.EMPTY_TRANSFORM, load); + + String newComment = "new_comment"; + // update table comment and column comment + // `col_1` int NOT NULL COMMENT 'id' , + // `col_2` STRING(255) NOT NULL DEFAULT 'hello world' COMMENT 'new_comment' , + // `col_3` STRING(255) NULL DEFAULT NULL COMMENT 'name' , + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + tableName, + TableChange.updateColumnType(new String[] {col_1.name()}, INT), + TableChange.updateColumnComment(new String[] {col_2.name()}, newComment)); + load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + + columns.clear(); + col_1 = + JdbcColumn.builder() + .withName(col_1.name()) + .withType(INT) + .withComment(col_1.comment()) + .withAutoIncrement(col_1.autoIncrement()) + .withNullable(col_1.nullable()) + .withDefaultValue(Literals.integerLiteral(0)) + .build(); + col_2 = + JdbcColumn.builder() + .withName(col_2.name()) + .withType(col_2.dataType()) + .withComment(newComment) + .withAutoIncrement(col_2.autoIncrement()) + .withNullable(col_2.nullable()) + .withDefaultValue(col_2.defaultValue()) + .build(); + columns.add(col_1); + columns.add(col_2); + columns.add(col_3); + assertionsTableInfo( + tableName, tableComment, columns, properties, null, Transforms.EMPTY_TRANSFORM, load); + + String newColName_1 = "new_col_1"; + // rename column + // update table comment and column comment + // `new_col_1` int NOT NULL COMMENT 'id' , + // `new_col_2` STRING(255) NOT NULL DEFAULT 'hello world' COMMENT 'new_comment' + // , + // `col_3` STRING(255) NULL DEFAULT NULL COMMENT 'name' , + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + tableName, + TableChange.renameColumn(new String[] {col_1.name()}, newColName_1)); + + load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + + columns.clear(); + col_1 = + JdbcColumn.builder() + .withName(newColName_1) + .withType(col_1.dataType()) + .withComment(col_1.comment()) + .withAutoIncrement(col_1.autoIncrement()) + .withNullable(col_1.nullable()) + .withDefaultValue(col_1.defaultValue()) + .build(); + columns.add(col_1); + columns.add(col_2); + columns.add(col_3); + assertionsTableInfo( + tableName, tableComment, columns, properties, null, Transforms.EMPTY_TRANSFORM, load); + } + + @Test + public void testAlterTableUpdateColumnDefaultValue() { + String tableName = RandomNameUtils.genRandomName("properties_table_"); + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + columns.add( + JdbcColumn.builder() + .withName("col_1") + .withType(Types.DecimalType.of(10, 2)) + .withComment("test_decimal") + .withNullable(false) + .withDefaultValue(Literals.decimalLiteral(Decimal.of("0.00", 10, 2))) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_2") + .withType(Types.LongType.get()) + .withNullable(false) + .withDefaultValue(Literals.longLiteral(0L)) + .withComment("long type") + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_3") + .withType(Types.TimestampType.withoutTimeZone()) + .withNullable(false) + .withComment("timestamp") + .withDefaultValue(Literals.timestampLiteral(LocalDateTime.parse("2013-01-01T00:00:00"))) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_4") + .withType(Types.StringType.get()) + .withNullable(false) + .withComment("STRING") + .withDefaultValue(Literals.of("hello", Types.StringType.get())) + .build()); + Map properties = new HashMap<>(); + + Index[] indexes = new Index[0]; + // create table + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + properties, + null, + Distributions.NONE, + indexes, + getSortOrders("col_2")); + + JdbcTable loaded = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + assertionsTableInfo( + tableName, tableComment, columns, properties, indexes, Transforms.EMPTY_TRANSFORM, loaded); + + TABLE_OPERATIONS.alterTable( + TEST_DB_NAME.toString(), + tableName, + TableChange.updateColumnDefaultValue( + new String[] {columns.get(0).name()}, + Literals.decimalLiteral(Decimal.of("1.23", 10, 2))), + TableChange.updateColumnDefaultValue( + new String[] {columns.get(1).name()}, Literals.longLiteral(1L)), + TableChange.updateColumnDefaultValue( + new String[] {columns.get(2).name()}, + Literals.timestampLiteral(LocalDateTime.parse("2024-04-01T00:00:00"))), + TableChange.updateColumnDefaultValue( + new String[] {columns.get(3).name()}, Literals.of("world", Types.StringType.get()))); + + loaded = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + Assertions.assertEquals( + Literals.decimalLiteral(Decimal.of("1.234", 10, 2)), loaded.columns()[0].defaultValue()); + Assertions.assertEquals(Literals.longLiteral(1L), loaded.columns()[1].defaultValue()); + Assertions.assertEquals( + Literals.timestampLiteral(LocalDateTime.parse("2024-04-01T00:00:00")), + loaded.columns()[2].defaultValue()); + Assertions.assertEquals( + Literals.of("world", Types.StringType.get()), loaded.columns()[3].defaultValue()); + } + + @Test + public void testCreateAndLoadTable() { + String tableName = RandomStringUtils.randomAlphabetic(16) + "_cl_table"; + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + columns.add( + JdbcColumn.builder() + .withName("col_1") + .withType(Types.DecimalType.of(10, 2)) + .withComment("test_decimal") + .withNullable(false) + .withDefaultValue(Literals.decimalLiteral(Decimal.of("0.00", 10, 2))) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_2") + .withType(Types.LongType.get()) + .withNullable(false) + .withDefaultValue(Literals.longLiteral(0L)) + .withComment("long type") + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_3") + .withType(Types.TimestampType.withoutTimeZone()) + .withNullable(false) + .withComment("timestamp") + .withDefaultValue(Literals.timestampLiteral(LocalDateTime.parse("2013-01-01T00:00:00"))) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_4") + .withType(Types.DateType.get()) + .withNullable(true) + .withComment("date") + .withDefaultValue(DEFAULT_VALUE_NOT_SET) + .build()); + Map properties = new HashMap<>(); + + Index[] indexes = new Index[0]; + // create table + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + properties, + null, + Distributions.NONE, + indexes, + getSortOrders("col_1")); + + JdbcTable loaded = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + assertionsTableInfo( + tableName, tableComment, columns, properties, indexes, Transforms.EMPTY_TRANSFORM, loaded); + } + + @Test + public void testCreateAllTypeTable() { + String tableName = RandomNameUtils.genRandomName("type_table_"); + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + columns.add( + JdbcColumn.builder() + .withName("col_1") + .withType(Types.ByteType.get()) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_2") + .withType(Types.ShortType.get()) + .withNullable(true) + .build()); + columns.add(JdbcColumn.builder().withName("col_3").withType(INT).withNullable(false).build()); + columns.add( + JdbcColumn.builder() + .withName("col_4") + .withType(Types.LongType.get()) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_5") + .withType(Types.FloatType.get()) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_6") + .withType(Types.DoubleType.get()) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_7") + .withType(Types.DateType.get()) + .withNullable(false) + .build()); + // columns.add( + // JdbcColumn.builder() + // .withName("col_8") + // .withType(Types.TimeType.get()) + // .withNullable(false) + // .build()); + columns.add( + JdbcColumn.builder() + .withName("col_9") + .withType(Types.TimestampType.withoutTimeZone()) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder().withName("col_10").withType(Types.DecimalType.of(10, 2)).build()); + columns.add( + JdbcColumn.builder().withName("col_11").withType(STRING).withNullable(false).build()); + columns.add( + JdbcColumn.builder() + .withName("col_12") + .withType(Types.FixedCharType.of(10)) + .withNullable(false) + .build()); + columns.add( + JdbcColumn.builder() + .withName("col_13") + .withType(Types.StringType.get()) + .withNullable(false) + .build()); + // columns.add( + // JdbcColumn.builder() + // .withName("col_14") + // .withType(Types.BinaryType.get()) + // .withNullable(false) + // .build()); + columns.add( + JdbcColumn.builder() + .withName("col_15") + .withType(Types.FixedCharType.of(10)) + .withNullable(false) + .build()); + + // create table + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + columns.toArray(new JdbcColumn[0]), + tableComment, + Collections.emptyMap(), + null, + Distributions.NONE, + Indexes.EMPTY_INDEXES, + getSortOrders("col_1")); + + JdbcTable load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), tableName); + assertionsTableInfo( + tableName, + tableComment, + columns, + Collections.emptyMap(), + null, + Transforms.EMPTY_TRANSFORM, + load); + } + + @Test + public void testCreateNotSupportTypeTable() { + String tableName = RandomNameUtils.genRandomName("type_table_"); + String tableComment = "test_comment"; + List columns = new ArrayList<>(); + List notSupportType = + Arrays.asList( + Types.FixedType.of(10), + Types.IntervalDayType.get(), + Types.IntervalYearType.get(), + Types.ListType.of(Types.DateType.get(), true), + Types.MapType.of(Types.StringType.get(), Types.IntegerType.get(), true), + Types.UnionType.of(Types.IntegerType.get()), + Types.StructType.of( + Types.StructType.Field.notNullField("col_1", Types.IntegerType.get()))); + + for (Type type : notSupportType) { + columns.clear(); + columns.add( + JdbcColumn.builder().withName("col_1").withType(type).withNullable(false).build()); + + JdbcColumn[] jdbcCols = columns.toArray(new JdbcColumn[0]); + Map emptyMap = Collections.emptyMap(); + IllegalArgumentException illegalArgumentException = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> { + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + tableName, + jdbcCols, + tableComment, + emptyMap, + null, + Distributions.NONE, + Indexes.EMPTY_INDEXES, + getSortOrders("col_1")); + }); + Assertions.assertTrue( + illegalArgumentException + .getMessage() + .contains( + String.format( + "Couldn't convert Gravitino type %s to ClickHouse type", + type.simpleString()))); + } + } + + @Test + public void testCreateMultipleTables() { + String test_table_1 = "test_table_1"; + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + test_table_1, + new JdbcColumn[] { + JdbcColumn.builder() + .withName("col_1") + .withType(Types.DecimalType.of(10, 2)) + .withComment("test_decimal") + .withNullable(false) + .withDefaultValue(Literals.decimalLiteral(Decimal.of("0.00"))) + .build() + }, + "test_comment", + null, + null, + Distributions.NONE, + Indexes.EMPTY_INDEXES, + getSortOrders("col_1")); + + String testDb = "test_db_2"; + + DATABASE_OPERATIONS.create(testDb, null, null); + List tables = TABLE_OPERATIONS.listTables(testDb); + Assertions.assertFalse(tables.contains(test_table_1)); + + String test_table_2 = "test_table_2"; + TABLE_OPERATIONS.create( + testDb, + test_table_2, + new JdbcColumn[] { + JdbcColumn.builder() + .withName("col_1") + .withType(Types.DecimalType.of(10, 2)) + .withComment("test_decimal") + .withNullable(false) + .withDefaultValue(Literals.decimalLiteral(Decimal.of("0.00"))) + .build() + }, + "test_comment", + null, + null, + Distributions.NONE, + Indexes.EMPTY_INDEXES, + getSortOrders("col_1")); + + tables = TABLE_OPERATIONS.listTables(TEST_DB_NAME.toString()); + Assertions.assertFalse(tables.contains(test_table_2)); + } + + @Test + public void testLoadTableDefaultProperties() { + String test_table_1 = RandomNameUtils.genRandomName("properties_table_"); + TABLE_OPERATIONS.create( + TEST_DB_NAME.toString(), + test_table_1, + new JdbcColumn[] { + JdbcColumn.builder() + .withName("col_1") + .withType(Types.DecimalType.of(10, 2)) + .withComment("test_decimal") + .withNullable(false) + .withDefaultValue(Literals.decimalLiteral(Decimal.of("0.0"))) + .build() + }, + "test_comment", + null, + null, + Distributions.NONE, + Indexes.EMPTY_INDEXES, + getSortOrders("col_1")); + JdbcTable load = TABLE_OPERATIONS.load(TEST_DB_NAME.toString(), test_table_1); + Assertions.assertEquals("MergeTree", load.properties().get(CLICKHOUSE_ENGINE_KEY)); + } +} diff --git a/catalogs/catalog-jdbc-clickhouse/src/test/resources/log4j2.properties b/catalogs/catalog-jdbc-clickhouse/src/test/resources/log4j2.properties new file mode 100644 index 00000000000..c3c0b1186b6 --- /dev/null +++ b/catalogs/catalog-jdbc-clickhouse/src/test/resources/log4j2.properties @@ -0,0 +1,70 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# Set to debug or trace if log4j initialization is failing +status = info + +# Name of the configuration +name = ConsoleLogConfig + +# Console appender configuration +appender.console.type = Console +appender.console.name = consoleLogger +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p [%t] %c{1}:%L - %m%n + +# Log files location +property.logPath = ${sys:gravitino.log.path:-build/catalog-jdbc-clickhouse-integration-test.log} + +# File appender configuration +appender.file.type = File +appender.file.name = fileLogger +appender.file.fileName = ${logPath} +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p [%t] %c{1}:%L - %m%n + +# Root logger level +rootLogger.level = info + +# Root logger referring to console and file appenders +rootLogger.appenderRef.stdout.ref = consoleLogger +rootLogger.appenderRef.file.ref = fileLogger + +# File appender configuration for testcontainers +appender.testcontainersFile.type = File +appender.testcontainersFile.name = testcontainersLogger +appender.testcontainersFile.fileName = build/testcontainers.log +appender.testcontainersFile.layout.type = PatternLayout +appender.testcontainersFile.layout.pattern = %d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n + +# Logger for testcontainers +logger.testcontainers.name = org.testcontainers +logger.testcontainers.level = debug +logger.testcontainers.appenderRef.file.ref = testcontainersLogger + +logger.tc.name = tc +logger.tc.level = debug +logger.tc.appenderRef.file.ref = testcontainersLogger + +logger.docker.name = com.github.dockerjava +logger.docker.level = warn +logger.docker.appenderRef.file.ref = testcontainersLogger + +logger.http.name = com.github.dockerjava.zerodep.shaded.org.apache.hc.client5.http.wire +logger.http.level = off \ No newline at end of file diff --git a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/JdbcCatalogOperations.java b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/JdbcCatalogOperations.java index 56490d475cb..e1855000add 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/JdbcCatalogOperations.java +++ b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/JdbcCatalogOperations.java @@ -427,8 +427,12 @@ public Table createTable( SortOrder[] sortOrders, Index[] indexes) throws NoSuchSchemaException, TableAlreadyExistsException { - Preconditions.checkArgument( - null == sortOrders || sortOrders.length == 0, "jdbc-catalog does not support sort orders"); + // clickhouse support sortOrders + if (!tableOperation.getClass().getSimpleName().equals("ClickHouseTableOperations")) { + Preconditions.checkArgument( + null == sortOrders || sortOrders.length == 0, + "jdbc-catalog does not support sort orders"); + } StringIdentifier identifier = Preconditions.checkNotNull( @@ -461,7 +465,8 @@ public Table createTable( resultProperties, partitioning, distribution, - indexes); + indexes, + sortOrders); return JdbcTable.builder() .withAuditInfo( diff --git a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcDatabaseOperations.java b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcDatabaseOperations.java index a1e47e032cf..ec52e19bf7e 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcDatabaseOperations.java +++ b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcDatabaseOperations.java @@ -69,6 +69,10 @@ public void create(String databaseName, String comment, Map prop } try (final Connection connection = getConnection()) { + if (connection.getCatalog().equals(databaseName)) { + connection.setCatalog(createSysDatabaseNameSet().iterator().next()); + } + JdbcConnectorUtils.executeUpdate( connection, generateCreateDatabaseSql(databaseName, comment, properties)); LOG.info("Finished creating database {}", databaseName); diff --git a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcTableOperations.java b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcTableOperations.java index 083ffaaaf86..aa96e7499f0 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcTableOperations.java +++ b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/JdbcTableOperations.java @@ -54,6 +54,7 @@ import org.apache.gravitino.rel.expressions.distributions.Distribution; import org.apache.gravitino.rel.expressions.distributions.Distributions; import org.apache.gravitino.rel.expressions.literals.Literals; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; import org.apache.gravitino.rel.expressions.transforms.Transform; import org.apache.gravitino.rel.expressions.transforms.Transforms; import org.apache.gravitino.rel.indexes.Index; @@ -102,12 +103,43 @@ public void create( Distribution distribution, Index[] indexes) throws TableAlreadyExistsException { + create( + databaseName, + tableName, + columns, + comment, + properties, + partitioning, + distribution, + indexes, + null); + } + + @Override + public void create( + String databaseName, + String tableName, + JdbcColumn[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + Index[] indexes, + SortOrder[] sortOrders) + throws TableAlreadyExistsException { LOG.info("Attempting to create table {} in database {}", tableName, databaseName); try (Connection connection = getConnection(databaseName)) { JdbcConnectorUtils.executeUpdate( connection, generateCreateTableSql( - tableName, columns, comment, properties, partitioning, distribution, indexes)); + tableName, + columns, + comment, + properties, + partitioning, + distribution, + indexes, + sortOrders)); LOG.info("Created table {} in database {}", tableName, databaseName); } catch (final SQLException se) { throw this.exceptionMapper.toGravitinoException(se); @@ -462,6 +494,24 @@ protected abstract String generateCreateTableSql( Distribution distribution, Index[] indexes); + protected String generateCreateTableSql( + String tableName, + JdbcColumn[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + Index[] indexes, + SortOrder[] sortOrders) { + if (sortOrders == null || sortOrders.length == 0) { + return generateCreateTableSql( + tableName, columns, comment, properties, partitioning, distribution, indexes); + } + + throw new UnsupportedOperationException( + "generateCreateTableSql with sortOrders defined is not supported"); + } + /** * The default implementation of this method is based on MySQL syntax, and if the catalog does not * support MySQL syntax, this method needs to be rewritten. diff --git a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/TableOperation.java b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/TableOperation.java index f22bd745365..c39b769882a 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/TableOperation.java +++ b/catalogs/catalog-jdbc-common/src/main/java/org/apache/gravitino/catalog/jdbc/operation/TableOperation.java @@ -32,6 +32,7 @@ import org.apache.gravitino.exceptions.TableAlreadyExistsException; import org.apache.gravitino.rel.TableChange; import org.apache.gravitino.rel.expressions.distributions.Distribution; +import org.apache.gravitino.rel.expressions.sorts.SortOrder; import org.apache.gravitino.rel.expressions.transforms.Transform; import org.apache.gravitino.rel.indexes.Index; @@ -72,6 +73,28 @@ void create( Index[] indexes) throws TableAlreadyExistsException; + /** + * @param databaseName The name of the database. + * @param tableName The name of the table. + * @param columns The columns of the table. + * @param comment The comment of the table. + * @param properties The properties of the table. + * @param partitioning The partitioning of the table. + * @param indexes The indexes of the table. + * @param sortOrders The sort orders of the table + */ + void create( + String databaseName, + String tableName, + JdbcColumn[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + Index[] indexes, + SortOrder[] sortOrders) + throws TableAlreadyExistsException; + /** * @param databaseName The name of the database. * @param tableName The name of the table. diff --git a/common/src/main/java/org/apache/gravitino/dto/rel/expressions/FuncExpressionDTO.java b/common/src/main/java/org/apache/gravitino/dto/rel/expressions/FuncExpressionDTO.java index abde934c2ac..afc7e7b93be 100644 --- a/common/src/main/java/org/apache/gravitino/dto/rel/expressions/FuncExpressionDTO.java +++ b/common/src/main/java/org/apache/gravitino/dto/rel/expressions/FuncExpressionDTO.java @@ -18,6 +18,7 @@ */ package org.apache.gravitino.dto.rel.expressions; +import java.util.Arrays; import lombok.EqualsAndHashCode; import org.apache.gravitino.rel.expressions.Expression; import org.apache.gravitino.rel.expressions.FunctionExpression; @@ -56,6 +57,15 @@ public ArgType argType() { return ArgType.FUNCTION; } + /** @return The string representation of the function expression. */ + @Override + public String toString() { + if (functionArgs.length == 0) { + return functionName + "()"; + } + return functionName + "(" + String.join(", ", Arrays.toString(functionArgs)) + ")"; + } + /** Builder for {@link FuncExpressionDTO}. */ public static class Builder { private String functionName; diff --git a/common/src/main/java/org/apache/gravitino/dto/util/DTOConverters.java b/common/src/main/java/org/apache/gravitino/dto/util/DTOConverters.java index 254de8c3245..858af793340 100644 --- a/common/src/main/java/org/apache/gravitino/dto/util/DTOConverters.java +++ b/common/src/main/java/org/apache/gravitino/dto/util/DTOConverters.java @@ -559,8 +559,9 @@ public static FunctionArg toFunctionArg(Expression expression) { if (Literals.NULL.equals(expression)) { return LiteralDTO.NULL; } + Object value = ((Literal) expression).value(); return LiteralDTO.builder() - .withValue((((Literal) expression).value().toString())) + .withValue(value == null ? null : value.toString()) .withDataType(((Literal) expression).dataType()) .build(); } else if (expression instanceof NamedReference.FieldReference) { diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 630550a7e55..cd45ebeaed3 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -72,6 +72,8 @@ opencsv = "2.3" metrics = "4.2.25" prometheus = "0.16.0" mysql = "8.0.23" +clickhouse = "0.7.1" +lz4 = "1.8.0" postgresql = "42.6.0" immutables-value = "2.10.0" selenium = "3.141.59" @@ -212,6 +214,7 @@ sqlite-jdbc = { group = "org.xerial", name = "sqlite-jdbc", version.ref = "sqlit commons-dbcp2 = { group = "org.apache.commons", name = "commons-dbcp2", version.ref = "commons-dbcp2" } testcontainers = { group = "org.testcontainers", name = "testcontainers", version.ref = "testcontainers" } testcontainers-mysql = { group = "org.testcontainers", name = "mysql", version.ref = "testcontainers" } +testcontainers-clickhouse = { group = "org.testcontainers", name = "clickhouse", version.ref = "testcontainers" } testcontainers-postgresql = { group = "org.testcontainers", name = "postgresql", version.ref = "testcontainers" } testcontainers-junit-jupiter = { group = "org.testcontainers", name = "junit-jupiter", version.ref = "testcontainers" } testcontainers-localstack = { group = "org.testcontainers", name = "localstack", version.ref = "testcontainers" } @@ -231,6 +234,8 @@ prometheus-client = { group = "io.prometheus", name = "simpleclient", version.re prometheus-dropwizard = { group = "io.prometheus", name = "simpleclient_dropwizard", version.ref = "prometheus" } prometheus-servlet = { group = "io.prometheus", name = "simpleclient_servlet", version.ref = "prometheus" } mysql-driver = { group = "mysql", name = "mysql-connector-java", version.ref = "mysql" } +clickhouse-driver = { group = "com.clickhouse", name = "clickhouse-jdbc", version.ref = "clickhouse" } +lz4-java = { group = "org.lz4", name = "lz4-java", version.ref = "lz4" } postgresql-driver = { group = "org.postgresql", name = "postgresql", version.ref = "postgresql" } minikdc = { group = "org.apache.hadoop", name = "hadoop-minikdc", version.ref = "hadoop-minikdc"} immutables-value = { module = "org.immutables:value", version.ref = "immutables-value" } diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ClickHouseContainer.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ClickHouseContainer.java new file mode 100644 index 00000000000..1ea2ae22e92 --- /dev/null +++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ClickHouseContainer.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.integration.test.container; + +import static java.lang.String.format; + +import com.google.common.collect.ImmutableSet; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.integration.test.util.TestDatabaseName; +import org.rnorth.ducttape.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Network; + +public class ClickHouseContainer extends BaseContainer { + public static final Logger LOG = LoggerFactory.getLogger(ClickHouseContainer.class); + + public static final String DEFAULT_IMAGE = "clickhouse:24.9.3.128"; + public static final String HOST_NAME = "gravitino-ci-clickhouse"; + public static final int CLICKHOUSE_PORT = 8123; + public static final String USER_NAME = "default"; + public static final String PASSWORD = "default"; + + public static Builder builder() { + return new Builder(); + } + + protected ClickHouseContainer( + String image, + String hostName, + Set ports, + Map extraHosts, + Map filesToMount, + Map envVars, + Optional network) { + super(image, hostName, ports, extraHosts, filesToMount, envVars, network); + } + + @Override + protected void setupContainer() { + super.setupContainer(); + withLogConsumer(new PrintingContainerLog(format("%-14s| ", "clickHouseContainer"))); + } + + @Override + public void start() { + super.start(); + Preconditions.check("clickHouse container startup failed!", checkContainerStatus(5)); + } + + @Override + protected boolean checkContainerStatus(int retryLimit) { + return true; + } + + public void createDatabase(TestDatabaseName testDatabaseName) { + String clickHouseJdbcUrl = + StringUtils.substring( + getJdbcUrl(testDatabaseName), 0, getJdbcUrl(testDatabaseName).lastIndexOf("/")); + + // change password for root user, Gravitino API must set password in catalog properties + try (Connection connection = + DriverManager.getConnection(clickHouseJdbcUrl, USER_NAME, getPassword()); + Statement statement = connection.createStatement()) { + + String query = String.format("CREATE DATABASE IF NOT EXISTS %s;", testDatabaseName); + // FIXME: String, which is used in SQL, can be unsafe + statement.execute(query); + LOG.info( + String.format("clickHouse container database %s has been created", testDatabaseName)); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + } + } + + public String getUsername() { + return USER_NAME; + } + + public String getPassword() { + return PASSWORD; + } + + public String getJdbcUrl() { + return format("jdbc:clickhouse://%s:%d", getContainerIpAddress(), CLICKHOUSE_PORT); + } + + public String getJdbcUrl(TestDatabaseName testDatabaseName) { + return format( + "jdbc:clickhouse://%s:%d/%s", getContainerIpAddress(), CLICKHOUSE_PORT, testDatabaseName); + } + + public String getDriverClassName(TestDatabaseName testDatabaseName) throws SQLException { + return DriverManager.getDriver(getJdbcUrl(testDatabaseName)).getClass().getName(); + } + + public static class Builder + extends BaseContainer.Builder { + + private Builder() { + this.image = DEFAULT_IMAGE; + this.hostName = HOST_NAME; + this.exposePorts = ImmutableSet.of(CLICKHOUSE_PORT); + } + + @Override + public ClickHouseContainer build() { + return new ClickHouseContainer( + image, hostName, exposePorts, extraHosts, filesToMount, envVars, network); + } + } +} diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ContainerSuite.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ContainerSuite.java index 14398b4b218..9c466c6ba36 100644 --- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ContainerSuite.java +++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/container/ContainerSuite.java @@ -68,6 +68,7 @@ public class ContainerSuite implements Closeable { private static volatile MySQLContainer mySQLContainer; private static volatile MySQLContainer mySQLVersion5Container; + private static volatile ClickHouseContainer clickHouseContainer; private static volatile Map pgContainerMap = new EnumMap<>(PGImageName.class); private static volatile OceanBaseContainer oceanBaseContainer; @@ -347,6 +348,33 @@ public void startMySQLVersion5Container(TestDatabaseName testDatabaseName) { } } + public void startClickHouseContainer(TestDatabaseName testDatabaseName) { + if (clickHouseContainer == null) { + synchronized (ContainerSuite.class) { + if (clickHouseContainer == null) { + initIfNecessary(); + // Start ClickHouse container + ClickHouseContainer.Builder clickHouseBuilder = + ClickHouseContainer.builder() + .withHostName("gravitino-ci-clickhouse") + .withEnvVars( + ImmutableMap.builder() + .put("CLICKHOUSE_PASSWORD", ClickHouseContainer.PASSWORD) + .build()) + .withExposePorts(ImmutableSet.of(ClickHouseContainer.CLICKHOUSE_PORT)) + .withNetwork(network); + + ClickHouseContainer container = closer.register(clickHouseBuilder.build()); + container.start(); + clickHouseContainer = container; + } + } + } + synchronized (ClickHouseContainer.class) { + clickHouseContainer.createDatabase(testDatabaseName); + } + } + public void startPostgreSQLContainer(TestDatabaseName testDatabaseName, PGImageName pgImageName) { if (!pgContainerMap.containsKey(pgImageName)) { synchronized (ContainerSuite.class) { @@ -678,4 +706,8 @@ public void close() throws IOException { LOG.error("Failed to close ContainerEnvironment", e); } } + + public ClickHouseContainer getClickHouseContainer() { + return clickHouseContainer; + } } diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/ITUtils.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/ITUtils.java index d7c099dc7ac..c39812b9a70 100644 --- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/ITUtils.java +++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/ITUtils.java @@ -44,6 +44,7 @@ import org.apache.gravitino.rel.Table; import org.apache.gravitino.rel.expressions.transforms.Transform; import org.apache.gravitino.rel.indexes.Index; +import org.apache.gravitino.rel.indexes.Index.IndexType; import org.apache.gravitino.rel.partitions.Partition; import org.junit.jupiter.api.Assertions; @@ -112,14 +113,34 @@ public static void assertionsTableInfo( Arrays.stream(indexes).collect(Collectors.toMap(Index::name, index -> index)); for (int i = 0; i < table.index().length; i++) { - Assertions.assertTrue(indexByName.containsKey(table.index()[i].name())); - Assertions.assertEquals( - indexByName.get(table.index()[i].name()).type(), table.index()[i].type()); - for (int j = 0; j < table.index()[i].fieldNames().length; j++) { - for (int k = 0; k < table.index()[i].fieldNames()[j].length; k++) { - Assertions.assertEquals( - indexByName.get(table.index()[i].name()).fieldNames()[j][k], - table.index()[i].fieldNames()[j][k]); + // some database don't support name primary key, such as clickhouse, mysql. + IndexType tableIndexType = table.index()[i].type(); + if (indexByName.get(table.index()[i].name()) == null) { + Assertions.assertTrue(table.index()[i].name().equalsIgnoreCase("PRIMARY")); + Assertions.assertEquals(tableIndexType, IndexType.PRIMARY_KEY); + + Index primKey = + indexByName.values().stream() + .filter(e -> IndexType.PRIMARY_KEY.equals(e.type())) + .findFirst() + .get(); + for (int j = 0; j < table.index()[i].fieldNames().length; j++) { + for (int k = 0; k < table.index()[i].fieldNames()[j].length; k++) { + Assertions.assertEquals( + primKey.fieldNames()[j][k], table.index()[i].fieldNames()[j][k]); + } + } + } else { + Assertions.assertTrue(indexByName.containsKey(table.index()[i].name())); + Assertions.assertEquals( + indexByName.get(table.index()[i].name()).type(), table.index()[i].type()); + + for (int j = 0; j < table.index()[i].fieldNames().length; j++) { + for (int k = 0; k < table.index()[i].fieldNames()[j].length; k++) { + Assertions.assertEquals( + indexByName.get(table.index()[i].name()).fieldNames()[j][k], + table.index()[i].fieldNames()[j][k]); + } } } } diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/TestDatabaseName.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/TestDatabaseName.java index 5f0ec3de4d8..96a67214b16 100644 --- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/TestDatabaseName.java +++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/TestDatabaseName.java @@ -102,4 +102,8 @@ public String toString() { return this.name().toLowerCase(); } }, + + CLICKHOUSE_CLICKHOUSE_ABSTRACT_IT, + CLICKHOUSE_CATALOG_CLICKHOUSE_IT, + CLICKHOUSE_AUDIT_CATALOG_CLICKHOUSE_IT, } diff --git a/settings.gradle.kts b/settings.gradle.kts index 75dd967c4e9..4ca885681dc 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -36,6 +36,7 @@ include( "catalogs:catalog-jdbc-common", "catalogs:catalog-jdbc-doris", "catalogs:catalog-jdbc-mysql", + "catalogs:catalog-jdbc-clickhouse", "catalogs:catalog-jdbc-postgresql", "catalogs:catalog-jdbc-oceanbase" ) diff --git a/web/web/src/app/metalakes/metalake/MetalakeTree.js b/web/web/src/app/metalakes/metalake/MetalakeTree.js index e6b6ea0c39a..4b16d437b85 100644 --- a/web/web/src/app/metalakes/metalake/MetalakeTree.js +++ b/web/web/src/app/metalakes/metalake/MetalakeTree.js @@ -64,6 +64,8 @@ const MetalakeTree = props => { return 'openmoji:iceberg' case 'jdbc-mysql': return 'devicon:mysql-wordmark' + case 'jdbc-clickhouse': + return 'devicon:clickhouse-wordmark' case 'jdbc-postgresql': return 'devicon:postgresql-wordmark' case 'jdbc-doris': diff --git a/web/web/src/lib/utils/initial.js b/web/web/src/lib/utils/initial.js index 6d6efe84b0d..315a18f5bc7 100644 --- a/web/web/src/lib/utils/initial.js +++ b/web/web/src/lib/utils/initial.js @@ -286,6 +286,34 @@ export const providers = [ } ] }, + { + label: 'ClickHouse', + value: 'jdbc-clickhouse', + defaultProps: [ + { + key: 'jdbc-driver', + value: '', + required: true, + description: 'e.g. com.clickhouse.jdbc.ClickHouseDriver' + }, + { + key: 'jdbc-url', + value: '', + required: true, + description: 'e.g. jdbc:clickhouse://localhost' + }, + { + key: 'jdbc-user', + value: '', + required: true + }, + { + key: 'jdbc-password', + value: '', + required: true + } + ] + }, { label: 'OceanBase', value: 'jdbc-oceanbase',