Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -2,6 +2,7 @@

import com.clickhouse.client.api.Client;
import com.clickhouse.client.api.ClientConfigProperties;
import org.apache.flink.runtime.util.EnvironmentInformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -16,6 +17,7 @@ public class ClickHouseClientConfig implements Serializable {
private final String password;
private final String database;
private final String tableName;
private final String fullProductName;
private Boolean supportDefault = null;

public ClickHouseClientConfig(String url, String username, String password, String database, String tableName) {
Expand All @@ -24,6 +26,7 @@ public ClickHouseClientConfig(String url, String username, String password, Stri
this.password = password;
this.database = database;
this.tableName = tableName;
this.fullProductName = String.format("Flink-ClickHouse-Sink/%s (fv:flink/%s, lv:scala/%s)", ClickHouseSinkVersion.getVersion(), EnvironmentInformation.getVersion(), EnvironmentInformation.getScalaVersion());
}

public Client createClient(String database) {
Expand All @@ -32,6 +35,7 @@ public Client createClient(String database) {
.setUsername(username)
.setPassword(password)
.setDefaultDatabase(database)
.setClientName(fullProductName)
.setOption(ClientConfigProperties.ASYNC_OPERATIONS.getKey(), "true")
.build();
return client;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
package org.apache.flink.connector.clickhouse.sink;

public class ClickHouseSinkVersion {
public static String getVersion() {
return "0.0.1";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -237,4 +237,69 @@ void SimplePOJODataTest() throws Exception {
int rows = executeJob(env, tableName);
Assertions.assertEquals(EXPECTED_ROWS, rows);
}

@Test
void ProductNameTest() throws Exception {
String tableName = "product_name_csv_covid";
String dropTable = String.format("DROP TABLE IF EXISTS `%s`.`%s`", getDatabase(), tableName);
ClickHouseServerForTests.executeSql(dropTable);
// create table
String tableSql = "CREATE TABLE `" + getDatabase() + "`.`" + tableName + "` (" +
"date Date," +
"location_key LowCardinality(String)," +
"new_confirmed Int32," +
"new_deceased Int32," +
"new_recovered Int32," +
"new_tested Int32," +
"cumulative_confirmed Int32," +
"cumulative_deceased Int32," +
"cumulative_recovered Int32," +
"cumulative_tested Int32" +
") " +
"ENGINE = MergeTree " +
"ORDER BY (location_key, date); ";
ClickHouseServerForTests.executeSql(tableSql);

final StreamExecutionEnvironment env = EmbeddedFlinkClusterForTests.getMiniCluster().getTestStreamEnvironment();
env.setParallelism(1);

ClickHouseClientConfig clickHouseClientConfig = new ClickHouseClientConfig(getServerURL(), getUsername(), getPassword(), getDatabase(), tableName);
ElementConverter<String, ClickHousePayload> convertorString = new ClickHouseConvertor<>(String.class);
// create sink
ClickHouseAsyncSink<String> csvSink = new ClickHouseAsyncSink<>(
convertorString,
5000,
2,
20000,
1024 * 1024,
5 * 1000,
1000,
clickHouseClientConfig
);
csvSink.setClickHouseFormat(ClickHouseFormat.CSV);

Path filePath = new Path("./src/test/resources/epidemiology_top_10000.csv.gz");

FileSource<String> source = FileSource
.forRecordStreamFormat(new TextLineInputFormat(), filePath)
.build();
// read csv data from file
DataStreamSource<String> lines = env.fromSource(
source,
WatermarkStrategy.noWatermarks(),
"GzipCsvSource"
);
lines.sinkTo(csvSink);
int rows = executeJob(env, tableName);
Assertions.assertEquals(EXPECTED_ROWS, rows);
ClickHouseServerForTests.executeSql("SYSTEM FLUSH LOGS");
if (ClickHouseServerForTests.isCloud())
Thread.sleep(2000);
// let's wait until data will be available in query log
String productName = ClickHouseServerForTests.extractProductName(ClickHouseServerForTests.getDatabase(), tableName);
String compareString = String.format("Flink-ClickHouse-Sink/%s (fv:flink/2.0.0, lv:scala/2.12)", ClickHouseSinkVersion.getVersion());

boolean isContains = productName.contains(compareString);
Assertions.assertTrue(isContains);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -86,9 +86,15 @@ public static String getURL() {
}
}

public static boolean isCloud() { return isCloud; }

public static void executeSql(String sql) throws ExecutionException, InterruptedException {
Client client = ClickHouseTestHelpers.getClient(host, port, isSSL, username, password);
client.execute(sql).get();
try {
client.execute(sql).get().close();
} catch (Exception e) {
throw new RuntimeException(e);
}
}

public static int countRows(String table) throws ExecutionException, InterruptedException {
Expand All @@ -97,6 +103,16 @@ public static int countRows(String table) throws ExecutionException, Interrupted
List<GenericRecord> countResult = client.queryAll(countSql);
return countResult.get(0).getInteger(1);
}
// http_user_agent
public static String extractProductName(String databaseName, String tableName) {
String extractProductName = String.format("SELECT http_user_agent, tables FROM clusterAllReplicas('default', system.query_log) WHERE type = 'QueryStart' AND query_kind = 'Insert' AND has(databases,'%s') LIMIT 100", databaseName);
Client client = ClickHouseTestHelpers.getClient(host, port, isSSL, username, password);
List<GenericRecord> userAgentResult = client.queryAll(extractProductName);
if (!userAgentResult.isEmpty()) {
return userAgentResult.get(0).getString(1);
}
throw new RuntimeException("Quest for system query failed");
}

public static TableSchema getTableSchema(String table) throws ExecutionException, InterruptedException {
Client client = ClickHouseTestHelpers.getClient(host, port, isSSL, username, password);
Expand Down
Loading