Skip to content

Commit 33b1fad

Browse files
authored
Merge pull request #2059 from ClickHouse/client_v2_fix_table_name
[client-v2] Fix table name wrapping in insert statement
2 parents ca81ec7 + a5fc84a commit 33b1fad

File tree

2 files changed

+14
-7
lines changed

2 files changed

+14
-7
lines changed

client-v2/src/main/java/com/clickhouse/client/api/Client.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -1534,7 +1534,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15341534

15351535
settings.setOption(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey(), format.name());
15361536
final InsertSettings finalSettings = settings;
1537-
final String sqlStmt = "INSERT INTO \"" + tableName + "\" FORMAT " + format.name();
1537+
final String sqlStmt = "INSERT INTO " + tableName + " FORMAT " + format.name();
15381538
finalSettings.serverSetting(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlStmt);
15391539
responseSupplier = () -> {
15401540
// Selecting some node

client-v2/src/test/java/com/clickhouse/client/insert/InsertTests.java

+13-6
Original file line numberDiff line numberDiff line change
@@ -254,12 +254,9 @@ public void insertRawData() throws Exception {
254254
assertEquals(records.size(), 1000);
255255
}
256256

257-
@Test(groups = { "integration" }, enabled = true)
258-
public void insertRawDataSimple() throws Exception {
259-
insertRawDataSimple(1000);
260-
}
261-
public void insertRawDataSimple(int numberOfRecords) throws Exception {
262-
final String tableName = "raw_data_table";
257+
@Test(groups = { "integration" }, dataProvider = "insertRawDataSimpleDataProvider", dataProviderClass = InsertTests.class)
258+
public void insertRawDataSimple(String tableName) throws Exception {
259+
// final String tableName = "raw_data_table";
263260
final String createSql = String.format("CREATE TABLE IF NOT EXISTS %s " +
264261
" (Id UInt32, event_ts Timestamp, name String, p1 Int64, p2 String) ENGINE = MergeTree() ORDER BY ()", tableName);
265262

@@ -271,6 +268,7 @@ public void insertRawDataSimple(int numberOfRecords) throws Exception {
271268
.setInputStreamCopyBufferSize(8198 * 2);
272269
ByteArrayOutputStream data = new ByteArrayOutputStream();
273270
PrintWriter writer = new PrintWriter(data);
271+
int numberOfRecords = 1000;
274272
for (int i = 0; i < numberOfRecords; i++) {
275273
writer.printf("%d\t%s\t%s\t%d\t%s\n", i, "2021-01-01 00:00:00", "name" + i, i, "p2");
276274
}
@@ -281,6 +279,15 @@ public void insertRawDataSimple(int numberOfRecords) throws Exception {
281279
assertEquals((int)response.getWrittenRows(), numberOfRecords );
282280
}
283281

282+
@DataProvider(name = "insertRawDataSimpleDataProvider")
283+
public static Object[][] insertRawDataSimpleDataProvider() {
284+
return new Object[][] {
285+
{"raw_data_table"},
286+
{"`raw_data_table`"},
287+
{"`" + ClickHouseServerForTest.getDatabase() + ".raw_data_table`"},
288+
};
289+
}
290+
284291
@Test(groups = { "integration" })
285292
public void testInsertMetricsOperationId() throws Exception {
286293
final String tableName = "insert_metrics_test";

0 commit comments

Comments
 (0)