Skip to content

Commit 552d8b3

Browse files
committed
Merge branch 'main' into clientv2_retries
2 parents e6be27b + 18d0ca1 commit 552d8b3

File tree

33 files changed

+1553
-544
lines changed

33 files changed

+1553
-544
lines changed

.github/workflows/build.yml

Lines changed: 4 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -146,45 +146,6 @@ jobs:
146146
- name: Test compressed native image
147147
run: ./clickhouse-jdbc/target/clickhouse-jdbc-bin
148148

149-
test-cli-client:
150-
runs-on: ubuntu-latest
151-
needs: compile
152-
timeout-minutes: 10
153-
name: CLI client + CH LTS
154-
steps:
155-
- name: Check out repository
156-
uses: actions/checkout@v4
157-
- name: Check out PR
158-
run: |
159-
git fetch --no-tags --prune --progress --no-recurse-submodules --depth=1 \
160-
origin pull/${{ github.event.inputs.pr }}/merge:merged-pr && git checkout merged-pr
161-
if: github.event.inputs.pr != ''
162-
- name: Install JDK 8 and Maven
163-
uses: actions/setup-java@v4
164-
with:
165-
distribution: "temurin"
166-
java-version: 8
167-
cache: "maven"
168-
- name: Install ClickHouse native command line
169-
run: |
170-
sudo apt-get update \
171-
&& sudo apt-get install -y apt-transport-https ca-certificates dirmngr \
172-
&& sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 \
173-
&& echo "deb https://packages.clickhouse.com/deb lts main" | sudo tee /etc/apt/sources.list.d/clickhouse.list \
174-
&& sudo apt-get update && sudo apt-get install -y clickhouse-client \
175-
&& clickhouse client --version
176-
- name: Test CLI client
177-
run: |
178-
mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-cli-client -DclickhouseVersion=$PREFERRED_LTS_VERSION -Dj8 -DskipUTs verify
179-
- name: Upload test results
180-
uses: actions/upload-artifact@v4
181-
if: failure()
182-
with:
183-
name: result ${{ github.job }}
184-
path: |
185-
**/target/failsafe-reports
186-
**/target/surefire-reports
187-
188149
test-java-client:
189150
runs-on: ubuntu-latest
190151
needs: compile
@@ -235,7 +196,7 @@ jobs:
235196
CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }}
236197
CLICKHOUSE_CLOUD_PASSWORD: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_PASSWORD_SMT }}
237198
run: |
238-
mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-cli-client,clickhouse-grpc-client,clickhouse-http-client -DclickhouseVersion=${{ matrix.clickhouse }} verify
199+
mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-http-client -DclickhouseVersion=${{ matrix.clickhouse }} verify
239200
- name: Upload test results
240201
uses: actions/upload-artifact@v4
241202
if: failure()
@@ -314,7 +275,7 @@ jobs:
314275
matrix:
315276
clickhouse: ["23.8", "24.3", "24.6", "latest", "cloud"]
316277
# here http, http_client and apache_http_client represent different value of http_connection_provider
317-
protocol: ["http", "http_client", "apache_http_client", "grpc"]
278+
protocol: ["http", "http_client", "apache_http_client"]
318279
fail-fast: false
319280
timeout-minutes: 15
320281
name: JDBC driver + CH ${{ matrix.clickhouse }} (${{ matrix.protocol }})
@@ -353,7 +314,7 @@ jobs:
353314
</toolchains>
354315
EOF
355316
- name: Install Java client
356-
run: mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-cli-client,clickhouse-grpc-client,clickhouse-http-client -DskipTests install
317+
run: mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-http-client -DskipTests install
357318
- name: Test JDBC driver
358319
env:
359320
CLICKHOUSE_CLOUD_HOST: ${{ secrets.INTEGRATIONS_TEAM_TESTS_CLOUD_HOST_SMT }}
@@ -469,7 +430,7 @@ jobs:
469430
java-version: 8
470431
cache: "maven"
471432
- name: Install Java client
472-
run: mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-cli-client,clickhouse-grpc-client,clickhouse-http-client -Dj8 -DskipTests install
433+
run: mvn --also-make --batch-mode --no-transfer-progress --projects clickhouse-http-client -Dj8 -DskipTests install
473434
- name: Test JDBC and R2DBC drivers
474435
run: |
475436
mvn --batch-mode --no-transfer-progress --projects clickhouse-jdbc,clickhouse-r2dbc -DclickhouseVersion=$PREFERRED_LTS_VERSION \
Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,15 @@
1+
/**
2+
* Declares com.clickhouse.client module.
3+
*/
4+
module com.clickhouse.client {
5+
exports com.clickhouse.client;
6+
exports com.clickhouse.client.config;
7+
8+
requires static org.dnsjava;
9+
10+
requires transitive com.clickhouse.data;
11+
12+
uses com.clickhouse.client.ClickHouseClient;
13+
uses com.clickhouse.client.ClickHouseDnsResolver;
14+
uses com.clickhouse.client.ClickHouseSslContextProvider;
15+
}

clickhouse-client/src/test/java/com/clickhouse/client/ClientIntegrationTest.java

Lines changed: 33 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -882,12 +882,14 @@ public void testQueryWithNoResult() throws ExecutionException, InterruptedExcept
882882

883883
@Test(groups = { "integration" })
884884
public void testQuery() {
885+
testQuery(10000);
886+
}
887+
public void testQuery(int totalRecords) {
885888
ClickHouseNode server = getServer();
886889

887890
try (ClickHouseClient client = getClient()) {
888891
// "select * from system.data_type_families"
889-
int limit = 10000;
890-
String sql = "select number, toString(number) from system.numbers limit " + limit;
892+
String sql = "select number, toString(number) from system.numbers limit " + totalRecords;
891893

892894
try (ClickHouseResponse response = newRequest(client, server)
893895
.format(ClickHouseFormat.RowBinaryWithNamesAndTypes)
@@ -909,20 +911,8 @@ public void testQuery() {
909911
}
910912
Assert.assertTrue(response.getInputStream().isClosed(),
911913
"Input stream should have been closed since there's no data");
912-
// int counter = 0;
913-
// for (ClickHouseValue value : response.values()) {
914-
// Assert.assertEquals(value.asString(), String.valueOf(index));
915-
// index += counter++ % 2;
916-
// }
917-
Assert.assertEquals(index, limit);
918-
// Thread.sleep(30000);
919-
/*
920-
* while (response.hasError()) { int index = 0; for (ClickHouseColumn c :
921-
* columns) { // RawValue v = response.getRawValue(index++); // String v =
922-
* response.getValue(index++, String.class) }
923-
*
924-
* } byte[] bytes = in.readAllBytes(); String str = new String(bytes);
925-
*/
914+
915+
Assert.assertEquals(index, totalRecords);
926916
} catch (Exception e) {
927917
Assert.fail("Query failed", e);
928918
}
@@ -1992,6 +1982,33 @@ public void testInsertWithCustomFormat() throws ClickHouseException {
19921982
}
19931983
}
19941984

1985+
@Test(groups = { "integration" })
1986+
public void testInsertRawDataSimple() throws Exception {
1987+
testInsertRawDataSimple(1000);
1988+
}
1989+
public void testInsertRawDataSimple(int numberOfRecords) throws Exception {
1990+
String tableName = "test_insert_raw_data_simple";
1991+
ClickHouseNode server = getServer();
1992+
sendAndWait(server, "DROP TABLE IF EXISTS " + tableName,
1993+
"CREATE TABLE IF NOT EXISTS "+ tableName + " (i Int16, f String) engine=MergeTree ORDER BY i");
1994+
try (ClickHouseClient client = getClient()) {
1995+
ClickHouseRequest.Mutation request = client.read(server).write().table(tableName).format(ClickHouseFormat.JSONEachRow);
1996+
ClickHouseConfig config = request.getConfig();
1997+
CompletableFuture<ClickHouseResponse> future;
1998+
try (ClickHousePipedOutputStream stream = ClickHouseDataStreamFactory.getInstance().createPipedOutputStream(config)) {
1999+
// start the worker thread which transfer data from the input into ClickHouse
2000+
future = request.data(stream.getInputStream()).execute();
2001+
for (int i = 0; i < numberOfRecords; i++) {
2002+
BinaryStreamUtils.writeBytes(stream, String.format("{\"i\": %s, \"\": \"JSON\"}", i).getBytes(StandardCharsets.UTF_8));
2003+
}
2004+
}
2005+
2006+
ClickHouseResponseSummary summary = future.get().getSummary();
2007+
Assert.assertEquals(summary.getWrittenRows(), numberOfRecords);
2008+
}
2009+
}
2010+
2011+
19952012
@Test(groups = { "integration" })
19962013
public void testInsertWithInputFunction() throws ClickHouseException {
19972014
ClickHouseNode server = getServer();
Lines changed: 76 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,76 @@
1+
package com.clickhouse.data;
2+
3+
import java.lang.invoke.MethodHandles;
4+
import java.lang.invoke.VarHandle;
5+
import java.nio.ByteOrder;
6+
import java.util.Arrays;
7+
8+
public final class ByteUtils {
9+
public boolean equals(byte[] a, int aFromIndex, int aToIndex, byte[] b, int bFromIndex, int bToIndex) {
10+
return Arrays.equals(a, aFromIndex, aToIndex, b, bFromIndex, bToIndex);
11+
}
12+
13+
public byte getInt8(byte[] bytes, int offset) {
14+
return (byte) byteHandle.get(bytes, offset);
15+
}
16+
17+
public void setInt8(byte[] bytes, int offset, byte value) {
18+
byteHandle.set(bytes, offset, value);
19+
}
20+
21+
public short getInt16(byte[] bytes, int offset) {
22+
return (short) shortHandle.get(bytes, offset);
23+
}
24+
25+
public void setInt16(byte[] bytes, int offset, short value) {
26+
shortHandle.set(bytes, offset, value);
27+
}
28+
29+
public int getInt32(byte[] bytes, int offset) {
30+
return (int) intHandle.get(bytes, offset);
31+
}
32+
33+
public void setInt32(byte[] bytes, int offset, int value) {
34+
intHandle.set(bytes, offset, value);
35+
}
36+
37+
public long getInt64(byte[] bytes, int offset) {
38+
return (long) longHandle.get(bytes, offset);
39+
}
40+
41+
public void setInt64(byte[] bytes, int offset, long value) {
42+
longHandle.set(bytes, offset, value);
43+
}
44+
45+
public float getFloat32(byte[] bytes, int offset) {
46+
return (float) floatHandle.get(bytes, offset);
47+
}
48+
49+
public void setFloat32(byte[] bytes, int offset, float value) {
50+
floatHandle.set(bytes, offset, value);
51+
}
52+
53+
public double getFloat64(byte[] bytes, int offset) {
54+
return (double) doubleHandle.get(bytes, offset);
55+
}
56+
57+
public void setFloat64(byte[] bytes, int offset, double value) {
58+
doubleHandle.set(bytes, offset, value);
59+
}
60+
61+
private final VarHandle byteHandle;
62+
private final VarHandle shortHandle;
63+
private final VarHandle intHandle;
64+
private final VarHandle longHandle;
65+
private final VarHandle floatHandle;
66+
private final VarHandle doubleHandle;
67+
68+
ByteUtils(ByteOrder byteOrder) {
69+
byteHandle = MethodHandles.arrayElementVarHandle(byte[].class);
70+
shortHandle = MethodHandles.byteArrayViewVarHandle(short[].class, byteOrder);
71+
intHandle = MethodHandles.byteArrayViewVarHandle(int[].class, byteOrder);
72+
longHandle = MethodHandles.byteArrayViewVarHandle(long[].class, byteOrder);
73+
floatHandle = MethodHandles.byteArrayViewVarHandle(float[].class, byteOrder);
74+
doubleHandle = MethodHandles.byteArrayViewVarHandle(double[].class, byteOrder);
75+
}
76+
}
Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,23 @@
1+
/**
2+
* Declares com.clickhouse.data module.
3+
*/
4+
module com.clickhouse.data {
5+
exports com.clickhouse.config;
6+
exports com.clickhouse.data;
7+
// exports com.clickhouse.data.cache;
8+
// exports com.clickhouse.data.format;
9+
// exports com.clickhouse.data.mapper;
10+
// exports com.clickhouse.data.stream;
11+
exports com.clickhouse.data.value;
12+
exports com.clickhouse.logging;
13+
14+
requires static java.logging;
15+
requires static com.google.gson;
16+
requires static com.github.benmanes.caffeine;
17+
requires static org.lz4.java;
18+
requires static org.slf4j;
19+
requires static org.roaringbitmap;
20+
21+
uses com.clickhouse.data.ClickHouseDataStreamFactory;
22+
uses com.clickhouse.logging.LoggerFactory;
23+
}

0 commit comments

Comments
 (0)