Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,43 @@

import org.apache.seatunnel.api.table.catalog.Column;
import org.apache.seatunnel.api.table.catalog.TablePath;
import org.apache.seatunnel.api.table.catalog.TableSchema;
import org.apache.seatunnel.connectors.seatunnel.clickhouse.catalog.ClickhouseTypeConverter;
import org.apache.seatunnel.connectors.seatunnel.common.util.CatalogUtil;

import java.util.HashSet;
import java.util.Set;

import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull;

public class ClickhouseCatalogUtil extends CatalogUtil {

private static final ThreadLocal<Set<String>> PRIMARY_KEY_COLUMNS =
ThreadLocal.withInitial(HashSet::new);

public static final ClickhouseCatalogUtil INSTANCE = new ClickhouseCatalogUtil();

@Override
public String getCreateTableSql(
String template,
String database,
String table,
TableSchema tableSchema,
String comment,
String optionsKey) {
Set<String> pkColumns = PRIMARY_KEY_COLUMNS.get();
pkColumns.clear();
if (tableSchema.getPrimaryKey() != null) {
pkColumns.addAll(tableSchema.getPrimaryKey().getColumnNames());
}
try {
return super.getCreateTableSql(
template, database, table, tableSchema, comment, optionsKey);
} finally {
pkColumns.clear();
}
}

public String columnToConnectorType(Column column) {
checkNotNull(column, "The column is required.");
String columnType;
Expand All @@ -38,6 +66,14 @@ public String columnToConnectorType(Column column) {
} else {
columnType = ClickhouseTypeConverter.INSTANCE.reconvert(column).getColumnType();
}

Set<String> pkColumns = PRIMARY_KEY_COLUMNS.get();
boolean isPrimaryKeyColumn = pkColumns != null && pkColumns.contains(column.getName());

if (column.isNullable() && !isUnsupportedNullableType(columnType) && !isPrimaryKeyColumn) {
columnType = "Nullable(" + columnType + ")";
}

return String.format(
"`%s` %s %s",
column.getName(),
Expand All @@ -49,6 +85,10 @@ public String columnToConnectorType(Column column) {
+ "'");
}

private static boolean isUnsupportedNullableType(String columnType) {
return columnType.startsWith("Map(") || columnType.startsWith("Array(");
}

public String getDropTableSql(TablePath tablePath, boolean ignoreIfNotExists) {
if (ignoreIfNotExists) {
return "DROP TABLE IF EXISTS "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,14 +99,16 @@ public void test() {
.build(),
"clickhouse test table",
ClickhouseSinkOptions.SAVE_MODE_CREATE_TEMPLATE.key());
// Primary key columns (id, age) should NOT be wrapped in Nullable
// because ClickHouse does not allow nullable columns in ORDER BY / PRIMARY KEY
Assertions.assertEquals(
createTableSql,
"CREATE TABLE IF NOT EXISTS `test1`.`test2` (\n"
+ " `id` Int64 ,`age` Int32 COMMENT 'test comment',\n"
+ " `name` String ,\n"
+ "`score` Int32 COMMENT '''N''-N',\n"
+ "`gender` Int8 ,\n"
+ "`create_time` Int64 \n"
+ " `name` Nullable(String) ,\n"
+ "`score` Nullable(Int32) COMMENT '\''N''-N',\n"
+ "`gender` Nullable(Int8) ,\n"
+ "`create_time` Nullable(Int64) \n"
+ ") ENGINE = MergeTree()\n"
+ "ORDER BY (`id`,`age`)\n"
+ "PRIMARY KEY (`id`,`age`)\n"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.junit.jupiter.api.Test;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

Expand All @@ -33,6 +34,8 @@ void returnsReconvertedTypeWhenSinkTypeNotNull() {
Column column = mock(Column.class);
when(column.getName()).thenReturn("col1");
when(column.getSinkType()).thenReturn("String");
when(column.isNullable()).thenReturn(false);
when(column.getComment()).thenReturn("");

String result = ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(column);

Expand All @@ -44,6 +47,8 @@ void returnsReconvertedTypeWhenSinkTypeIsNull() {
Column column = mock(Column.class);
when(column.getName()).thenReturn("col1");
when(column.getDataType()).thenReturn((SeaTunnelDataType) BasicType.INT_TYPE);
when(column.isNullable()).thenReturn(false);
when(column.getComment()).thenReturn("");

String result = ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(column);

Expand All @@ -56,9 +61,44 @@ void returnsReconvertedTypeWhenTypesNotNull() {
when(column.getName()).thenReturn("col1");
when(column.getDataType()).thenReturn((SeaTunnelDataType) BasicType.INT_TYPE);
when(column.getSinkType()).thenReturn("String");
when(column.isNullable()).thenReturn(false);
when(column.getComment()).thenReturn("");

String result = ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(column);

assertEquals("`col1` String ", result);
}

@Test
void wrapsTypeWithNullableWhenColumnIsNullable() {
Column column = mock(Column.class);
when(column.getName()).thenReturn("col1");
when(column.getSinkType()).thenReturn("String");
when(column.isNullable()).thenReturn(true);
when(column.getComment()).thenReturn("");

String result = ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(column);

assertEquals("`col1` Nullable(String) ", result);
}

@Test
void escapesSingleQuoteAndBackslashInComment() {
Column column = mock(Column.class);
when(column.getName()).thenReturn("col1");
when(column.getSinkType()).thenReturn("String");
when(column.isNullable()).thenReturn(false);
when(column.getComment()).thenReturn("O'Reilly \\ path");

String result = ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(column);

assertEquals("`col1` String COMMENT 'O''Reilly \\\\ path'", result);
}

@Test
void throwsExceptionWhenColumnIsNull() {
assertThrows(
NullPointerException.class,
() -> ClickhouseCatalogUtil.INSTANCE.columnToConnectorType(null));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ public void clickhouseRecreateSchemaAndCustom(TestContainer container) throws Ex
String tableName = "default.sink_table_for_schema";
Container.ExecResult execResult =
container.executeJob("/clickhouse_with_recreate_schema_and_custom.conf");
Assertions.assertEquals(0, execResult.getExitCode());
Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStdout());
Assertions.assertEquals(101, countData(tableName));
dropTable(tableName);
}
Expand Down
Loading