From 6fd2928deeffca46d77d0600676dc027ac125fb0 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Mon, 23 Mar 2026 23:13:05 -0700 Subject: [PATCH 1/6] Implemented reading ordinary errors --- .../api/internal/HttpAPIClientHelper.java | 142 +++++++++++------- .../clickhouse/client/HttpTransportTests.java | 24 +++ 2 files changed, 109 insertions(+), 57 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index 76e2dec93..6bca14de2 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -84,8 +84,10 @@ import java.net.URISyntaxException; import java.net.URLEncoder; import java.net.UnknownHostException; +import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; import java.util.Base64; import java.util.Collection; import java.util.Collections; @@ -104,6 +106,7 @@ import java.util.function.BiConsumer; import java.util.function.Function; import java.util.regex.Pattern; +import java.util.stream.Stream; public class HttpAPIClientHelper { @@ -351,75 +354,100 @@ public CloseableHttpClient createHttpClient(boolean initSslContext, Map 0 ? readClickHouseError(httpResponse.getEntity(), serverCode, queryId, httpResponse.getCode()) : + readNotClickHouseError(httpResponse.getEntity(), queryId, httpResponse.getCode()); + } catch (Exception e) { + LOG.error("Failed to read error message", e); + String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; + return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); + } + } + + private ServerException readNotClickHouseError(HttpEntity httpEntity, String queryId, int httpCode) { + + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + + String msg = null; + try { + InputStream body = httpEntity.getContent(); + int msgLen = body.read(buffer, 0, buffer.length - 2); + msg = new String(buffer, 0, msgLen, StandardCharsets.UTF_8); + } catch (Exception e) { + LOG.warn("Failed to read error message (queryId = " + queryId + ")", e); + } + + String errormsg = msg == null ? "unknown server error" : msg; + return new ServerException(ServerException.CODE_UNKNOWN, errormsg + " (transport error: " + httpCode +")", httpCode, queryId); + } + + private static ServerException readClickHouseError(HttpEntity httpEntity, int serverCode, String queryId, int httpCode) throws Exception { + InputStream body = httpEntity.getContent(); + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + byte[] lookUpStr = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode).getBytes(StandardCharsets.UTF_8); + StringBuilder msgBuilder = new StringBuilder(); + boolean found = false; + while (true) { + int rBytes = -1; + try { + rBytes = body.read(buffer); + } catch (ClientException e) { + // Invalid LZ4 Magic + if (body instanceof ClickHouseLZ4InputStream) { + ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; + body = stream.getInputStream(); + byte[] headerBuffer = stream.getHeaderBuffer(); + System.arraycopy(headerBuffer, 0, buffer, 0, headerBuffer.length); + rBytes = headerBuffer.length; } + } + if (rBytes == -1) { + break; + } - for (int i = 0; i < rBytes; i++) { - if (buffer[i] == lookUpStr[0]) { - found = true; - for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { - if (buffer[i + j] != lookUpStr[j]) { - found = false; - break; - } - } - if (found) { - msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); + for (int i = 0; i < rBytes; i++) { + if (buffer[i] == lookUpStr[0]) { + found = true; + for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { + if (buffer[i + j] != lookUpStr[j]) { + found = false; break; } } - } - - if (found) { - break; + if (found) { + msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); + break; + } } } - while (true) { - int rBytes = body.read(buffer); - if (rBytes == -1) { - break; - } - msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); + if (found) { + break; } + } - String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") - .replaceAll("\\\\/", "/"); - if (msg.trim().isEmpty()) { - msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; + while (true) { + int rBytes = body.read(buffer); + if (rBytes == -1) { + break; } - return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); - } catch (Exception e) { - LOG.error("Failed to read error message", e); - String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; - return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); + msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); + } + + String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") + .replaceAll("\\\\/", "/"); + if (msg.trim().isEmpty()) { + msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpCode + ")"; } + return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpCode, queryId); } private static final long POOL_VENT_TIMEOUT = 10000L; @@ -536,7 +564,7 @@ private ClassicHttpResponse doPostRequest(Map requestConfig, Htt throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings."); } else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) { try { - throw readError(httpResponse); + throw readError(req, httpResponse); } finally { httpResponse.close(); } @@ -742,7 +770,7 @@ private HttpEntity wrapResponseEntity(HttpEntity httpEntity, int httpStatus, Map } // data compression - if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED)) { + if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED || httpStatus == HttpStatus.SC_NOT_FOUND)) { int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); return new LZ4Entity(httpEntity, useHttpCompression, true, false, buffSize, true, lz4Factory); } diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index fc2d13a86..e91bebdf4 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -1226,6 +1226,30 @@ public void testMultiPartRequest() { } } + @Test(groups = {"integration"}) + public void testNotFoundError() { + if (isCloud()) { + return; // not needed + } + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint("http://" + node.getHost() + ":" + node.getPort() + "/some-path") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + + try (Client client = clientBuilder.build()) { + client.queryAll("select 1"); + fail("Exception expected"); + } catch (ClientException e) { + Assert.assertTrue(e.getCause().getMessage().startsWith("There is no handle /some-path?")); + } + + } + protected Client.Builder newClient() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); boolean isSecure = isCloud(); From 5eb35556dbb18e9d2be529326cc7edd6b86dfc61 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Tue, 24 Mar 2026 13:45:38 -0700 Subject: [PATCH 2/6] Added tests for 401, 403, 404 http errors --- .../clickhouse/client/HttpTransportTests.java | 156 ++++++++++++++++++ 1 file changed, 156 insertions(+) diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index e91bebdf4..3140f2d7e 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -36,6 +36,7 @@ import org.testng.annotations.Test; import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.net.InetAddress; import java.net.Socket; import java.nio.ByteBuffer; @@ -54,6 +55,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import java.util.zip.GZIPOutputStream; import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; import static org.testng.Assert.fail; @@ -1250,6 +1252,160 @@ public void testNotFoundError() { } + @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") + public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withBody(responseBody)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected to contain '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusErrorBodyDataProvider") + public static Object[][] testHttpStatusErrorBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: invalid credentials for user default", "invalid credentials"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: user default has no access to this operation", "no access"}, + {HttpStatus.SC_NOT_FOUND, "Not found: requested endpoint does not exist", "requested endpoint"} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusWithoutBodyDataProvider") + public void testHttpStatusErrorsWithoutBody(int httpStatus) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withStatus(httpStatus)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains("unknown server error"), + "Expected unknown error message for empty body, but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusWithoutBodyDataProvider") + public static Object[][] testHttpStatusWithoutBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED}, + {HttpStatus.SC_FORBIDDEN}, + {HttpStatus.SC_NOT_FOUND} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusCompressedBodyDataProvider") + public void testHttpStatusErrorsWithHttpCompression(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withHeader(HttpHeaders.CONTENT_ENCODING, "gzip") + .withBody(gzip(responseBody))) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .useHttpCompression(true) + .compressServerResponse(true) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected compressed body part '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusCompressedBodyDataProvider") + public static Object[][] testHttpStatusCompressedBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: token is expired", "token is expired"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: policy denies this query", "policy denies"}, + {HttpStatus.SC_NOT_FOUND, "Not found: route does not exist", "route does not exist"} + }; + } + + private static byte[] gzip(String value) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { + gzipOutputStream.write(value.getBytes(StandardCharsets.UTF_8)); + } + return out.toByteArray(); + } + + private static ServerException findServerException(Throwable throwable) { + Throwable current = throwable; + while (current != null) { + if (current instanceof ServerException) { + return (ServerException) current; + } + current = current.getCause(); + } + return null; + } + protected Client.Builder newClient() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); boolean isSecure = isCloud(); From a480c75f5f8f6a3c4c168fe21c187655bad94eef Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Wed, 25 Mar 2026 14:23:45 -0700 Subject: [PATCH 3/6] Added more information to IOException while reading data --- .../com/clickhouse/data/ClickHouseColumn.java | 4 + .../internal/AbstractBinaryFormatReader.java | 64 ++-- .../clickhouse/client/HttpTransportTests.java | 285 +++++++++++++++++- 3 files changed, 331 insertions(+), 22 deletions(-) diff --git a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java index 3baff8680..0c368f72b 100644 --- a/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java +++ b/clickhouse-data/src/main/java/com/clickhouse/data/ClickHouseColumn.java @@ -952,6 +952,10 @@ public String getColumnName() { return columnName; } + public String getColumnIndexAndName() { + return (columnIndex + 1) + " (`" + columnName + "`)"; + } + public String getOriginalTypeName() { return originalTypeName; } diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index 8abcdc65b..dbb1cee33 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -2,6 +2,7 @@ import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.DataTransferException; import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.internal.DataTypeConverter; @@ -73,6 +74,8 @@ public abstract class AbstractBinaryFormatReader implements ClickHouseBinaryForm private Map[] convertions; private boolean hasNext = true; private boolean initialState = true; // reader is in initial state, no records have been read yet + private long row = -1; // before first row + private long lastNextCallTs; // for exception to detect slow reader protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings querySettings, TableSchema schema,BinaryStreamReader.ByteBufferAllocator byteBufferAllocator, Map> defaultTypeHintMap) { this.input = inputStream; @@ -92,6 +95,7 @@ protected AbstractBinaryFormatReader(InputStream inputStream, QuerySettings quer setSchema(schema); } this.dataTypeConverter = DataTypeConverter.INSTANCE; // singleton while no need to customize conversion + this.lastNextCallTs = System.currentTimeMillis(); } protected Object[] currentRecord; @@ -181,6 +185,7 @@ protected boolean readRecord(Object[] record) throws IOException { return false; } + row++; boolean firstColumn = true; for (int i = 0; i < columns.length; i++) { try { @@ -191,12 +196,12 @@ protected boolean readRecord(Object[] record) throws IOException { record[i] = null; } firstColumn = false; - } catch (EOFException e) { - if (firstColumn) { + } catch (IOException e) { + if (e instanceof EOFException && firstColumn) { endReached(); return false; } - throw e; + throw new IOException(recordReadExceptionMsg(columns[i].getColumnIndexAndName()), e); } } return true; @@ -238,35 +243,52 @@ protected void readNextRecord() { } } catch (IOException e) { endReached(); - throw new ClientException("Failed to read next row", e); + throw new ClientException(recordReadExceptionMsg(), e); } } + private long timeSinceLastNext() { + return System.currentTimeMillis() - lastNextCallTs; + } + + private String recordReadExceptionMsg() { + return recordReadExceptionMsg(null); + } + + private String recordReadExceptionMsg(String column) { + return "Reading " + (column != null ? "column " + column + " in " : "") + + " row " + row + " (time since last next call " + timeSinceLastNext() + ")"; + } + @Override public Map next() { if (!hasNext) { return null; } - if (!nextRecordEmpty) { - Object[] tmp = currentRecord; - currentRecord = nextRecord; - nextRecord = tmp; - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - try { - if (readRecord(currentRecord)) { - readNextRecord(); - return new RecordWrapper(currentRecord, schema); - } else { - currentRecord = null; - return null; + try { + if (!nextRecordEmpty) { + Object[] tmp = currentRecord; + currentRecord = nextRecord; + nextRecord = tmp; + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + try { + if (readRecord(currentRecord)) { + readNextRecord(); + return new RecordWrapper(currentRecord, schema); + } else { + currentRecord = null; + return null; + } + } catch (IOException e) { + endReached(); + throw new ClientException(recordReadExceptionMsg(), e); } - } catch (IOException e) { - endReached(); - throw new ClientException("Failed to read row", e); } + } finally { + lastNextCallTs = System.currentTimeMillis(); } } diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index 3140f2d7e..91518c6c3 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -9,6 +9,7 @@ import com.clickhouse.client.api.ServerException; import com.clickhouse.client.api.command.CommandResponse; import com.clickhouse.client.api.command.CommandSettings; +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.enums.Protocol; import com.clickhouse.client.api.enums.ProxyType; import com.clickhouse.client.api.insert.InsertResponse; @@ -26,6 +27,7 @@ import com.github.tomakehurst.wiremock.core.WireMockConfiguration; import com.github.tomakehurst.wiremock.http.Fault; import com.github.tomakehurst.wiremock.http.trafficlistener.WiremockNetworkTrafficListener; +import org.apache.hc.core5.http.ConnectionClosedException; import org.apache.hc.core5.http.ConnectionRequestTimeoutException; import org.apache.hc.core5.http.HttpHeaders; import org.apache.hc.core5.http.HttpStatus; @@ -606,7 +608,9 @@ public void testServerSettings() { } static { - System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); + if (Boolean.getBoolean("test.debug")) { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); + } } @Test(groups = { "integration" }) @@ -1252,6 +1256,176 @@ public void testNotFoundError() { } + @Test(groups = {"integration"}) + public void testSmallNetworkBufferDoesNotBreakColumnDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + Assert.assertTrue(validBody.length > 3, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - 5); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading truncated binary stream"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testChunkedResponsePrematureEndIsReported() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT 10"; + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader(HttpHeaders.TRANSFER_ENCODING, "chunked") + .withFault(Fault.MALFORMED_RESPONSE_CHUNK)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, null, 30, + "Expected failure when reading malformed chunked response"); + ConnectionClosedException connectionClosedException = findCause(thrown, ConnectionClosedException.class); + boolean hasChunkedPrematureCloseSignature = containsMessageInCauseChain(thrown, + "closing chunk expected", + "premature end of chunk coded message body", + "failed to read header"); + Assert.assertTrue(connectionClosedException != null || hasChunkedPrematureCloseSignature, + "Expected chunked/premature-close failure signature, but was: " + thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailCorruptedStreamFailsDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 100_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading binary stream truncated at tail"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailStreamFailureReportsPositiveTimeSinceLastNextCall() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize) + .build(); + QueryResponse response = client.query(query, querySettings).get(60, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + final int[] rowsRead = new int[] {0}; + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + while (true) { + if (rowsRead[0] >= 5) { + try { + Thread.sleep(25); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + if (reader.next() == null) { + return; + } + rowsRead[0]++; + } + }); + + Assert.assertTrue(rowsRead[0] >= 5, + "Expected to read at least a few rows before failure, but read " + rowsRead[0]); + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + + String elapsedTimeMessage = findFirstMessageInCauseChain(thrown, "time since last next call"); + Assert.assertNotNull(elapsedTimeMessage, + "Expected elapsed-time fragment in failure message chain, but was: " + thrown); + + java.util.regex.Matcher matcher = Pattern.compile("time since last next call (\\d+)\\)") + .matcher(elapsedTimeMessage); + Assert.assertTrue(matcher.find(), + "Expected elapsed-time fragment in message: " + elapsedTimeMessage); + long elapsedSinceLastNext = Long.parseLong(matcher.group(1)); + Assert.assertTrue(elapsedSinceLastNext > 0, + "Expected positive elapsed time since last next call, but was " + elapsedSinceLastNext); + } + } finally { + mockServer.stop(); + } + } + @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { if (isCloud()) { @@ -1387,6 +1561,56 @@ public static Object[][] testHttpStatusCompressedBodyDataProvider() { }; } + private byte[] fetchBinaryPayload(String query, int networkBufferSize, int timeoutSec) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newClient() + .useHttpCompression(false) + .compressServerResponse(false) + .setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)) + .build(); + QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS)) { + return readAllBytes(response.getInputStream()); + } + } + + private Throwable assertBinaryDecodeFails(WireMockServer mockServer, String query, Integer networkBufferSize, + int timeoutSec, String assertionMessage) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize).build()) { + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + try (QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + readAllRows(reader); + } + }); + Assert.assertNotNull(thrown, assertionMessage); + return thrown; + } + } + + private Client.Builder newMockServerClient(int port, Integer networkBufferSize) { + Client.Builder builder = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", port, false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1") + .useHttpCompression(false) + .compressServerResponse(false); + if (networkBufferSize != null) { + builder.setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)); + } + return builder; + } + + private static void readAllRows(ClickHouseBinaryFormatReader reader) { + while (reader.next() != null) { + reader.getInteger(1); + reader.getLong(2); + reader.getString(3); + } + } + private static byte[] gzip(String value) throws Exception { ByteArrayOutputStream out = new ByteArrayOutputStream(); try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { @@ -1395,6 +1619,16 @@ private static byte[] gzip(String value) throws Exception { return out.toByteArray(); } + private static byte[] readAllBytes(java.io.InputStream inputStream) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + byte[] buffer = new byte[8192]; + int read; + while ((read = inputStream.read(buffer)) != -1) { + out.write(buffer, 0, read); + } + return out.toByteArray(); + } + private static ServerException findServerException(Throwable throwable) { Throwable current = throwable; while (current != null) { @@ -1406,6 +1640,55 @@ private static ServerException findServerException(Throwable throwable) { return null; } + private static T findCause(Throwable throwable, Class clazz) { + Throwable current = throwable; + while (current != null) { + if (clazz.isInstance(current)) { + return clazz.cast(current); + } + current = current.getCause(); + } + return null; + } + + private static boolean containsMessageInCauseChain(Throwable throwable, String... parts) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null) { + String lower = message.toLowerCase(); + for (String part : parts) { + if (lower.contains(part.toLowerCase())) { + return true; + } + } + } + current = current.getCause(); + } + return false; + } + + private static String findFirstMessageInCauseChain(Throwable throwable, String part) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null && message.contains(part)) { + return message; + } + current = current.getCause(); + } + return null; + } + + private static void assertBinaryReadFailureContainsColumnName(Throwable thrown) { + thrown.printStackTrace(); + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + } + protected Client.Builder newClient() { ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); boolean isSecure = isCloud(); From 10e0cd3c391dee980b64305a52cc0edc103a7dda Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 2 Apr 2026 13:38:03 -0700 Subject: [PATCH 4/6] Added compressed stream unwrapping. --- .../internal/AbstractBinaryFormatReader.java | 3 +- .../api/internal/HttpAPIClientHelper.java | 2112 +++++----- .../clickhouse/client/HttpTransportTests.java | 3473 ++++++++--------- 3 files changed, 2799 insertions(+), 2789 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java index dbb1cee33..e5892748d 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/data_formats/internal/AbstractBinaryFormatReader.java @@ -2,7 +2,6 @@ import com.clickhouse.client.api.ClientConfigProperties; import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.DataTransferException; import com.clickhouse.client.api.DataTypeUtils; import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; import com.clickhouse.client.api.internal.DataTypeConverter; @@ -257,7 +256,7 @@ private String recordReadExceptionMsg() { private String recordReadExceptionMsg(String column) { return "Reading " + (column != null ? "column " + column + " in " : "") - + " row " + row + " (time since last next call " + timeSinceLastNext() + ")"; + + " row " + (row + 1) + " (time since last next call " + timeSinceLastNext() + ")"; } @Override diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index 0d9ffa28b..ddc41d798 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -1,1050 +1,1062 @@ -package com.clickhouse.client.api.internal; - -import com.clickhouse.client.ClickHouseSslContextProvider; -import com.clickhouse.client.api.ClickHouseException; -import com.clickhouse.client.api.Client; -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.ClientFaultCause; -import com.clickhouse.client.api.ClientMisconfigurationException; -import com.clickhouse.client.api.ConnectionInitiationException; -import com.clickhouse.client.api.ConnectionReuseStrategy; -import com.clickhouse.client.api.DataTransferException; -import com.clickhouse.client.api.ServerException; -import com.clickhouse.client.api.enums.ProxyType; -import com.clickhouse.client.api.http.ClickHouseHttpProto; -import com.clickhouse.client.api.transport.Endpoint; -import com.clickhouse.data.ClickHouseFormat; -import net.jpountz.lz4.LZ4Factory; -import org.apache.commons.compress.compressors.CompressorStreamFactory; -import org.apache.hc.client5.http.ConnectTimeoutException; -import org.apache.hc.client5.http.classic.methods.HttpPost; -import org.apache.hc.client5.http.config.ConnectionConfig; -import org.apache.hc.client5.http.config.RequestConfig; -import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder; -import org.apache.hc.client5.http.entity.mime.MultipartPartBuilder; -import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.client5.http.impl.io.BasicHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; -import org.apache.hc.client5.http.io.HttpClientConnectionManager; -import org.apache.hc.client5.http.io.ManagedHttpClientConnection; -import org.apache.hc.client5.http.protocol.HttpClientContext; -import org.apache.hc.client5.http.socket.ConnectionSocketFactory; -import org.apache.hc.client5.http.socket.LayeredConnectionSocketFactory; -import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory; -import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; -import org.apache.hc.core5.http.ClassicHttpResponse; -import org.apache.hc.core5.http.ConnectionRequestTimeoutException; -import org.apache.hc.core5.http.ContentType; -import org.apache.hc.core5.http.Header; -import org.apache.hc.core5.http.HttpEntity; -import org.apache.hc.core5.http.HttpHeaders; -import org.apache.hc.core5.http.HttpHost; -import org.apache.hc.core5.http.HttpRequest; -import org.apache.hc.core5.http.HttpStatus; -import org.apache.hc.core5.http.NoHttpResponseException; -import org.apache.hc.core5.http.config.CharCodingConfig; -import org.apache.hc.core5.http.config.Http1Config; -import org.apache.hc.core5.http.config.RegistryBuilder; -import org.apache.hc.core5.http.impl.io.DefaultHttpResponseParserFactory; -import org.apache.hc.core5.http.io.SocketConfig; -import org.apache.hc.core5.http.io.entity.ByteArrayEntity; -import org.apache.hc.core5.http.io.entity.EntityTemplate; -import org.apache.hc.core5.http.protocol.HttpContext; -import org.apache.hc.core5.io.CloseMode; -import org.apache.hc.core5.io.IOCallback; -import org.apache.hc.core5.net.URIBuilder; -import org.apache.hc.core5.pool.ConnPoolControl; -import org.apache.hc.core5.pool.PoolConcurrencyPolicy; -import org.apache.hc.core5.pool.PoolReusePolicy; -import org.apache.hc.core5.util.TimeValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.net.ssl.HostnameVerifier; -import javax.net.ssl.SNIHostName; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLException; -import javax.net.ssl.SSLParameters; -import javax.net.ssl.SSLSocket; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Method; -import java.net.ConnectException; -import java.net.InetSocketAddress; -import java.net.NoRouteToHostException; -import java.net.Socket; -import java.net.SocketTimeoutException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.net.UnknownHostException; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.security.NoSuchAlgorithmException; -import java.util.ArrayList; -import java.util.Base64; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Properties; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.regex.Pattern; -import java.util.stream.Stream; - -public class HttpAPIClientHelper { - - public static final String KEY_STATEMENT_PARAMS = "statement_params"; - - private static final Logger LOG = LoggerFactory.getLogger(HttpAPIClientHelper.class); - - private static final int ERROR_BODY_BUFFER_SIZE = 1024; // Error messages are usually small - - private final String DEFAULT_HTTP_COMPRESSION_ALGO = "lz4"; - - private static final Pattern PATTERN_HEADER_VALUE_ASCII = Pattern.compile( - "\\p{Graph}+(?:[ ]\\p{Graph}+)*"); - - private final CloseableHttpClient httpClient; - - private String proxyAuthHeaderValue; - - private final Set defaultRetryCauses; - - private final String defaultUserAgent; - private final Object metricsRegistry; - - ConnPoolControl poolControl; - - LZ4Factory lz4Factory; - - public HttpAPIClientHelper(Map configuration, Object metricsRegistry, boolean initSslContext, LZ4Factory lz4Factory) { - this.metricsRegistry = metricsRegistry; - this.httpClient = createHttpClient(initSslContext, configuration); - this.lz4Factory = lz4Factory; - assert this.lz4Factory != null; - - boolean usingClientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration); - boolean usingServerCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(configuration); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(configuration); - - LOG.debug("client compression: {}, server compression: {}, http compression: {}", usingClientCompression, usingServerCompression, useHttpCompression); - - defaultRetryCauses = new HashSet<>(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(configuration)); - if (defaultRetryCauses.contains(ClientFaultCause.None)) { - defaultRetryCauses.removeIf(c -> c != ClientFaultCause.None); - } - - this.defaultUserAgent = buildDefaultUserAgent(); - } - - /** - * Creates or returns default SSL context. - * - * @return SSLContext - */ - public SSLContext createSSLContext(Map configuration) { - SSLContext sslContext; - try { - sslContext = SSLContext.getDefault(); - } catch (NoSuchAlgorithmException e) { - throw new ClientException("Failed to create default SSL context", e); - } - ClickHouseSslContextProvider sslContextProvider = ClickHouseSslContextProvider.getProvider(); - String trustStorePath = (String) configuration.get(ClientConfigProperties.SSL_TRUST_STORE.getKey()); - if (trustStorePath != null) { - try { - sslContext = sslContextProvider.getSslContextFromKeyStore( - trustStorePath, - (String) configuration.get(ClientConfigProperties.SSL_KEY_STORE_PASSWORD.getKey()), - (String) configuration.get(ClientConfigProperties.SSL_KEYSTORE_TYPE.getKey()) - ); - } catch (SSLException e) { - throw new ClientMisconfigurationException("Failed to create SSL context from a keystore", e); - } - } else if (configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) != null || - configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()) != null || - configuration.get(ClientConfigProperties.SSL_KEY.getKey()) != null) { - - try { - sslContext = sslContextProvider.getSslContextFromCerts( - (String) configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()), - (String) configuration.get(ClientConfigProperties.SSL_KEY.getKey()), - (String) configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) - ); - } catch (SSLException e) { - throw new ClientMisconfigurationException("Failed to create SSL context from certificates", e); - } - } - return sslContext; - } - - private static final long CONNECTION_INACTIVITY_CHECK = 5000L; - - private ConnectionConfig createConnectionConfig(Map configuration) { - ConnectionConfig.Builder connConfig = ConnectionConfig.custom(); - - ClientConfigProperties.CONNECTION_TTL.applyIfSet(configuration, (t) -> connConfig.setTimeToLive(t, TimeUnit.MILLISECONDS)); - ClientConfigProperties.CONNECTION_TIMEOUT.applyIfSet(configuration, (t) -> connConfig.setConnectTimeout(t, TimeUnit.MILLISECONDS)); - connConfig.setValidateAfterInactivity(CONNECTION_INACTIVITY_CHECK, TimeUnit.MILLISECONDS); // non-configurable for now - - return connConfig.build(); - } - - private HttpClientConnectionManager basicConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { - RegistryBuilder registryBuilder = RegistryBuilder.create(); - registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory()); - registryBuilder.register("https", sslConnectionSocketFactory); - - BasicHttpClientConnectionManager connManager = new BasicHttpClientConnectionManager(registryBuilder.build()); - connManager.setConnectionConfig(createConnectionConfig(configuration)); - connManager.setSocketConfig(socketConfig); - - return connManager; - } - - private HttpClientConnectionManager poolConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { - PoolingHttpClientConnectionManagerBuilder connMgrBuilder = PoolingHttpClientConnectionManagerBuilder.create() - .setPoolConcurrencyPolicy(PoolConcurrencyPolicy.LAX); - - ConnectionReuseStrategy connectionReuseStrategy = ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getOrDefault(configuration); - switch (connectionReuseStrategy) { - case LIFO: - connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.LIFO); - break; - case FIFO: - connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.FIFO); - break; - default: - throw new ClientMisconfigurationException("Unknown connection reuse strategy: " + connectionReuseStrategy); - } - LOG.debug("Connection reuse strategy: {}", connectionReuseStrategy); - - connMgrBuilder.setDefaultConnectionConfig(createConnectionConfig(configuration)); - connMgrBuilder.setMaxConnTotal(Integer.MAX_VALUE); // as we do not know how many routes we will have - ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.applyIfSet(configuration, connMgrBuilder::setMaxConnPerRoute); - - int networkBufferSize = ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getOrDefault(configuration); - MeteredManagedHttpClientConnectionFactory connectionFactory = new MeteredManagedHttpClientConnectionFactory( - Http1Config.custom() - .setBufferSize(networkBufferSize) - .build(), - CharCodingConfig.DEFAULT, - DefaultHttpResponseParserFactory.INSTANCE); - - connMgrBuilder.setConnectionFactory(connectionFactory); - connMgrBuilder.setSSLSocketFactory(sslConnectionSocketFactory); - connMgrBuilder.setDefaultSocketConfig(socketConfig); - PoolingHttpClientConnectionManager phccm = connMgrBuilder.build(); - poolControl = phccm; - if (metricsRegistry != null) { - try { - String mGroupName = ClientConfigProperties.METRICS_GROUP_NAME.getOrDefault(configuration); - Class micrometerLoader = getClass().getClassLoader().loadClass("com.clickhouse.client.api.metrics.MicrometerLoader"); - Method applyMethod = micrometerLoader.getDeclaredMethod("applyPoolingMetricsBinder", Object.class, String.class, PoolingHttpClientConnectionManager.class); - applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, phccm); - - applyMethod = micrometerLoader.getDeclaredMethod("applyConnectionMetricsBinder", Object.class, String.class, MeteredManagedHttpClientConnectionFactory.class); - applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, connectionFactory); - } catch (Exception e) { - LOG.error("Failed to register metrics", e); - } - } - return phccm; - } - - public CloseableHttpClient createHttpClient(boolean initSslContext, Map configuration) { - // Top Level builders - HttpClientBuilder clientBuilder = HttpClientBuilder.create(); - SSLContext sslContext = initSslContext ? createSSLContext(configuration) : null; - LayeredConnectionSocketFactory sslConnectionSocketFactory; - if (sslContext != null) { - String socketSNI = (String)configuration.get(ClientConfigProperties.SSL_SOCKET_SNI.getKey()); - if (socketSNI != null && !socketSNI.trim().isEmpty()) { - sslConnectionSocketFactory = new CustomSSLConnectionFactory(socketSNI, sslContext, (hostname, session) -> true); - } else { - sslConnectionSocketFactory = new SSLConnectionSocketFactory(sslContext); - } - } else { - sslConnectionSocketFactory = new DummySSLConnectionSocketFactory(); - } - // Socket configuration - SocketConfig.Builder soCfgBuilder = SocketConfig.custom(); - ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.applyIfSet(configuration, - (t) -> soCfgBuilder.setSoTimeout(t, TimeUnit.MILLISECONDS)); - - ClientConfigProperties.SOCKET_RCVBUF_OPT.applyIfSet(configuration, - soCfgBuilder::setRcvBufSize); - - ClientConfigProperties.SOCKET_SNDBUF_OPT.applyIfSet(configuration, - soCfgBuilder::setSndBufSize); - - ClientConfigProperties.SOCKET_LINGER_OPT.applyIfSet(configuration, - (v) -> soCfgBuilder.setSoLinger(v, TimeUnit.SECONDS)); - - ClientConfigProperties.SOCKET_TCP_NO_DELAY_OPT.applyIfSet(configuration, - soCfgBuilder::setTcpNoDelay); - // Proxy - String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey()); - Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey()); - HttpHost proxy = null; - if (proxyHost != null && proxyPort != null) { - proxy = new HttpHost(proxyHost, proxyPort); - } - - String proxyTypeVal = (String) configuration.get(ClientConfigProperties.PROXY_TYPE.getKey()); - ProxyType proxyType = proxyTypeVal == null ? null : ProxyType.valueOf(proxyTypeVal); - if (proxyType == ProxyType.HTTP) { - clientBuilder.setProxy(proxy); - String proxyUser = (String) configuration.get(ClientConfigProperties.PROXY_USER.getKey()); - String proxyPassword = (String) configuration.get(ClientConfigProperties.PROXY_PASSWORD.getKey()); - if (proxyUser != null && proxyPassword != null) { - proxyAuthHeaderValue = "Basic " + Base64.getEncoder().encodeToString((proxyUser + ":" + proxyPassword).getBytes(StandardCharsets.UTF_8)); - } - - } else if (proxyType == ProxyType.SOCKS) { - soCfgBuilder.setSocksProxyAddress(new InetSocketAddress(proxyHost, proxyPort)); - } - - boolean disableCookies = !((Boolean)ClientConfigProperties.HTTP_SAVE_COOKIES.getOrDefault(configuration)); - if (disableCookies) { - clientBuilder.disableCookieManagement(); - } - SocketConfig socketConfig = soCfgBuilder.build(); - - // Connection manager - if (ClientConfigProperties.CONNECTION_POOL_ENABLED.getOrDefault(configuration)) { - clientBuilder.setConnectionManager(poolConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); - } else { - clientBuilder.setConnectionManager(basicConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); - } - Long keepAliveTimeout = ClientConfigProperties.HTTP_KEEP_ALIVE_TIMEOUT.getOrDefault(configuration); - if (keepAliveTimeout != null && keepAliveTimeout > 0) { - clientBuilder.setKeepAliveStrategy((response, context) -> TimeValue.ofMilliseconds(keepAliveTimeout)); - } - - clientBuilder.disableContentCompression(); // will handle ourselves - - return clientBuilder.build(); - } - -// private static final String ERROR_CODE_PREFIX_PATTERN = "Code: %d. DB::Exception:"; - private static final String ERROR_CODE_PREFIX_PATTERN = "%d. DB::Exception:"; - - - /** - * Reads status line and if error tries to parse response body to get server error message. - * - * @param httpResponse - HTTP response - * @return exception object with server code - */ - public Exception readError(HttpPost req, ClassicHttpResponse httpResponse) { - final Header serverQueryIdHeader = httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); - final Header clientQueryIdHeader = req.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); - final Header queryHeader = Stream.of(serverQueryIdHeader, clientQueryIdHeader).filter(Objects::nonNull).findFirst().orElse(null); - final String queryId = queryHeader == null ? "" : queryHeader.getValue(); - int serverCode = getHeaderInt(httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE), 0); - try { - - - return serverCode > 0 ? readClickHouseError(httpResponse.getEntity(), serverCode, queryId, httpResponse.getCode()) : - readNotClickHouseError(httpResponse.getEntity(), queryId, httpResponse.getCode()); - } catch (Exception e) { - LOG.error("Failed to read error message", e); - String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; - return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); - } - } - - private ServerException readNotClickHouseError(HttpEntity httpEntity, String queryId, int httpCode) { - - byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; - - String msg = null; - try { - InputStream body = httpEntity.getContent(); - int msgLen = body.read(buffer, 0, buffer.length - 2); - msg = new String(buffer, 0, msgLen, StandardCharsets.UTF_8); - } catch (Exception e) { - LOG.warn("Failed to read error message (queryId = " + queryId + ")", e); - } - - String errormsg = msg == null ? "unknown server error" : msg; - return new ServerException(ServerException.CODE_UNKNOWN, errormsg + " (transport error: " + httpCode +")", httpCode, queryId); - } - - private static ServerException readClickHouseError(HttpEntity httpEntity, int serverCode, String queryId, int httpCode) throws Exception { - InputStream body = httpEntity.getContent(); - byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; - byte[] lookUpStr = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode).getBytes(StandardCharsets.UTF_8); - StringBuilder msgBuilder = new StringBuilder(); - boolean found = false; - while (true) { - int rBytes = -1; - try { - rBytes = body.read(buffer); - } catch (ClientException e) { - // Invalid LZ4 Magic - if (body instanceof ClickHouseLZ4InputStream) { - ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; - body = stream.getInputStream(); - byte[] headerBuffer = stream.getHeaderBuffer(); - System.arraycopy(headerBuffer, 0, buffer, 0, headerBuffer.length); - rBytes = headerBuffer.length; - } - } - if (rBytes == -1) { - break; - } - - for (int i = 0; i < rBytes; i++) { - if (buffer[i] == lookUpStr[0]) { - found = true; - for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { - if (buffer[i + j] != lookUpStr[j]) { - found = false; - break; - } - } - if (found) { - msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); - break; - } - } - } - - if (found) { - break; - } - } - - while (true) { - int rBytes = body.read(buffer); - if (rBytes == -1) { - break; - } - msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); - } - - String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") - .replaceAll("\\\\/", "/"); - if (msg.trim().isEmpty()) { - msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpCode + ")"; - } - return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpCode, queryId); - } - - private static final long POOL_VENT_TIMEOUT = 10000L; - private final AtomicLong timeToPoolVent = new AtomicLong(0); - - private void doPoolVent() { - if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) { - timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT); - poolControl.closeExpired(); - } - } - - private HttpContext createRequestHttpContext(Map requestConfig) { - HttpClientContext context = HttpClientContext.create(); - Number responseTimeout = ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getOrDefault(requestConfig); - Number connectionReqTimeout = ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getOrDefault(requestConfig); - RequestConfig reqHttpConf = RequestConfig.custom() - .setResponseTimeout(responseTimeout.longValue(), TimeUnit.MILLISECONDS) - .setConnectionRequestTimeout(connectionReqTimeout.longValue(), TimeUnit.MILLISECONDS) - .build(); - context.setRequestConfig(reqHttpConf); - - return context; - } - - private URI createRequestURI(Endpoint server, Map requestConfig, boolean addParameters) { - URI uri; - try { - URIBuilder uriBuilder = new URIBuilder(server.getURI()); - addRequestParams(requestConfig, uriBuilder::addParameter); - - if (addParameters) { - addStatementParams(requestConfig, uriBuilder::addParameter); - } - - uri = uriBuilder.optimize().build(); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - return uri; - } - - private HttpPost createPostRequest(URI uri, Map requestConfig) { - HttpPost req = new HttpPost(uri); -// req.setVersion(new ProtocolVersion("HTTP", 1, 0)); // to disable chunk transfer encoding - addHeaders(req, requestConfig); - return req; - } - - public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, - String body) throws Exception { - - final URI uri = createRequestURI(server, requestConfig, true); - final HttpPost req = createPostRequest(uri, requestConfig); - final String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; - - HttpEntity httpEntity = new ByteArrayEntity(body.getBytes(StandardCharsets.UTF_8.name()), CONTENT_TYPE, contentEncoding); - req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); - - return doPostRequest(requestConfig, req); - } - - public ClassicHttpResponse executeMultiPartRequest(Endpoint server, Map requestConfig, String sqlQuery) throws Exception { - - requestConfig.put(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey(), false); - - final URI uri = createRequestURI(server, requestConfig, false); - final HttpPost req = createPostRequest(uri, requestConfig); - - MultipartEntityBuilder multipartEntityBuilder = MultipartEntityBuilder.create(); - addStatementParams(requestConfig, multipartEntityBuilder::addTextBody); - multipartEntityBuilder.addTextBody(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlQuery); - - - HttpEntity httpEntity = multipartEntityBuilder.build(); - req.setHeader(HttpHeaders.CONTENT_TYPE, httpEntity.getContentType()); // set proper content type with generated boundary value - req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); - - return doPostRequest(requestConfig, req); - - - } - - public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, - IOCallback writeCallback) throws Exception { - - final URI uri = createRequestURI(server, requestConfig, true); - final HttpPost req = createPostRequest(uri, requestConfig); - String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; - req.setEntity(wrapRequestEntity( - new EntityTemplate(-1, CONTENT_TYPE, contentEncoding , writeCallback), - requestConfig)); - - return doPostRequest(requestConfig, req); - } - - private ClassicHttpResponse doPostRequest(Map requestConfig, HttpPost req) throws Exception { - - doPoolVent(); - - ClassicHttpResponse httpResponse = null; - HttpContext context = createRequestHttpContext(requestConfig); - try { - httpResponse = httpClient.executeOpen(null, req, context); - - httpResponse.setEntity(wrapResponseEntity(httpResponse.getEntity(), - httpResponse.getCode(), - requestConfig)); - - if (httpResponse.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) { - throw new ClientMisconfigurationException("Proxy authentication required. Please check your proxy settings."); - } else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) { - httpResponse.close(); - throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings."); - } else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) { - try { - throw readError(req, httpResponse); - } finally { - httpResponse.close(); - } - } - return httpResponse; - - } catch (UnknownHostException e) { - closeQuietly(httpResponse); - LOG.warn("Host '{}' unknown", req.getAuthority()); - throw e; - } catch (ConnectException | NoRouteToHostException e) { - closeQuietly(httpResponse); - LOG.warn("Failed to connect to '{}': {}", req.getAuthority(), e.getMessage()); - throw e; - } catch (Exception e) { - closeQuietly(httpResponse); - LOG.debug("Failed to execute request to '{}': {}", req.getAuthority(), e.getMessage(), e); - throw e; - } - } - - public static void closeQuietly(ClassicHttpResponse httpResponse) { - if (httpResponse != null) { - try { - httpResponse.close(); - } catch (IOException e) { - LOG.warn("Failed to close response"); - } - } - } - - private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8"); - - private void addHeaders(HttpPost req, Map requestConfig) { - setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType()); - if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) { - setHeader( - req, - ClickHouseHttpProto.HEADER_FORMAT, - ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name()); - } - if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { - setHeader( - req, - ClickHouseHttpProto.HEADER_QUERY_ID, - (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey())); - } - setHeader( - req, - ClickHouseHttpProto.HEADER_DATABASE, - ClientConfigProperties.DATABASE.getOrDefault(requestConfig)); - - if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) { - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_USER, - ClientConfigProperties.USER.getOrDefault(requestConfig)); - setHeader( - req, - ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, - "on"); - } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) { - String user = ClientConfigProperties.USER.getOrDefault(requestConfig); - String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig); - if (password == null) { - password = ""; - } - // Use as-is, no encoding allowed - req.addHeader( - HttpHeaders.AUTHORIZATION, - "Basic " + Base64.getEncoder().encodeToString( - (user + ":" + password).getBytes(StandardCharsets.UTF_8))); - } else { - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_USER, - ClientConfigProperties.USER.getOrDefault(requestConfig)); - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_PASSWORD, - ClientConfigProperties.PASSWORD.getOrDefault(requestConfig)); - } - if (proxyAuthHeaderValue != null) { - req.addHeader( - HttpHeaders.PROXY_AUTHORIZATION, - proxyAuthHeaderValue); - } - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); - - if (useHttpCompression) { - if (serverCompression) { - setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); - } - - if (clientCompression && !appCompressedData) { - setHeader(req, HttpHeaders.CONTENT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); - } - } - - for (String key : requestConfig.keySet()) { - if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { - Object val = requestConfig.get(key); - if (val != null) { - setHeader( - req, - key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), - String.valueOf(val)); - } - } - } - - // Special cases - if (req.containsHeader(HttpHeaders.AUTHORIZATION) - && (req.containsHeader(ClickHouseHttpProto.HEADER_DB_USER) || - req.containsHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD))) - { - // user has set auth header for purpose, lets remove ours - req.removeHeaders(ClickHouseHttpProto.HEADER_DB_USER); - req.removeHeaders(ClickHouseHttpProto.HEADER_DB_PASSWORD); - } - - // -- keep last - correctUserAgentHeader(req, requestConfig); - } - - private void addRequestParams(Map requestConfig, BiConsumer consumer) { - if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { - consumer.accept(ClickHouseHttpProto.QPARAM_QUERY_ID, requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()).toString()); - } - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - - if (useHttpCompression) { - // enable_http_compression make server react on http header - // for client side compression Content-Encoding should be set - // for server side compression Accept-Encoding should be set - consumer.accept("enable_http_compression", "1"); - } else { - if (serverCompression) { - consumer.accept("compress", "1"); - } - if (clientCompression) { - consumer.accept("decompress", "1"); - } - } - - Collection sessionRoles = ClientConfigProperties.SESSION_DB_ROLES.getOrDefault(requestConfig); - if (!(sessionRoles == null || sessionRoles.isEmpty())) { - sessionRoles.forEach(r -> consumer.accept(ClickHouseHttpProto.QPARAM_ROLE, r)); - } - - for (String key : requestConfig.keySet()) { - if (key.startsWith(ClientConfigProperties.SERVER_SETTING_PREFIX)) { - Object val = requestConfig.get(key); - if (val != null) { - consumer.accept(key.substring(ClientConfigProperties.SERVER_SETTING_PREFIX.length()), String.valueOf(requestConfig.get(key))); - } - } - } - } - - private void addStatementParams(Map requestConfig, BiConsumer consumer) { - if (requestConfig.containsKey(KEY_STATEMENT_PARAMS)) { - Map params = (Map) requestConfig.get(KEY_STATEMENT_PARAMS); - params.forEach((k, v) -> consumer.accept("param_" + k, String.valueOf(v))); - } - } - - private HttpEntity wrapRequestEntity(HttpEntity httpEntity, Map requestConfig) { - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); - - LOG.debug("wrapRequestEntity: client compression: {}, http compression: {}, content encoding: {}", - clientCompression, useHttpCompression, httpEntity.getContentEncoding()); - - if (httpEntity.getContentEncoding() != null && !appCompressedData) { - // http header is set and data is not compressed - return new CompressedEntity(httpEntity, false, CompressorStreamFactory.getSingleton()); - } else if (clientCompression && !appCompressedData) { - int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); - return new LZ4Entity(httpEntity, useHttpCompression, false, true, - buffSize, false, lz4Factory); - } else { - return httpEntity; - } - } - - private HttpEntity wrapResponseEntity(HttpEntity httpEntity, int httpStatus, Map requestConfig) { - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - - LOG.debug("wrapResponseEntity: server compression: {}, http compression: {}, content encoding: {}", - serverCompression, useHttpCompression, httpEntity.getContentEncoding()); - - if (httpEntity.getContentEncoding() != null) { - // http compressed response - return new CompressedEntity(httpEntity, true, CompressorStreamFactory.getSingleton()); - } - - // data compression - if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED || httpStatus == HttpStatus.SC_NOT_FOUND)) { - int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); - return new LZ4Entity(httpEntity, useHttpCompression, true, false, buffSize, true, lz4Factory); - } - - return httpEntity; - } - - public static int getHeaderInt(Header header, int defaultValue) { - return getHeaderVal(header, defaultValue, Integer::parseInt); - } - - private static final Set RESPONSE_HEADER_WHITELIST = new HashSet<>(Arrays.asList( - ClickHouseHttpProto.HEADER_QUERY_ID, - ClickHouseHttpProto.HEADER_SRV_SUMMARY, - ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME, - ClickHouseHttpProto.HEADER_DATABASE, - ClickHouseHttpProto.HEADER_DB_USER - )); - - /** - * Collects whitelisted response headers from an HTTP response into a map. - * - * @param response the HTTP response - * @return unmodifiable map of header name to header value for whitelisted headers present in the response - */ - public static Map collectResponseHeaders(ClassicHttpResponse response) { - Map headers = new HashMap<>(); - for (String name : RESPONSE_HEADER_WHITELIST) { - Header header = response.getFirstHeader(name); - if (header != null) { - headers.put(name, header.getValue()); - } - } - return Collections.unmodifiableMap(headers); - } - - public static String getHeaderVal(Header header, String defaultValue) { - return getHeaderVal(header, defaultValue, Function.identity()); - } - - public static T getHeaderVal(Header header, T defaultValue, Function converter) { - if (header == null) { - return defaultValue; - } - - return converter.apply(header.getValue()); - } - - public boolean shouldRetry(Throwable ex, Map requestSettings) { - List retryCauses = ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(requestSettings); - - if (retryCauses.contains(ClientFaultCause.None)) { - return false; - } - - if (ex instanceof NoHttpResponseException - || ex.getCause() instanceof NoHttpResponseException) { - return retryCauses.contains(ClientFaultCause.NoHttpResponse); - } - - if (ex instanceof ConnectException - || ex instanceof ConnectTimeoutException - || ex.getCause() instanceof ConnectException - || ex.getCause() instanceof ConnectTimeoutException) { - return retryCauses.contains(ClientFaultCause.ConnectTimeout); - } - - if (ex instanceof ConnectionRequestTimeoutException - || ex.getCause() instanceof ConnectionRequestTimeoutException) { - return retryCauses.contains(ClientFaultCause.ConnectionRequestTimeout); - } - - if (ex instanceof SocketTimeoutException - || ex.getCause() instanceof SocketTimeoutException) { - return retryCauses.contains(ClientFaultCause.SocketTimeout); - } - - // there are some db retryable error codes - if (ex instanceof ServerException || ex.getCause() instanceof ServerException) { - ServerException se = (ServerException) ex; - return se.isRetryable() && retryCauses.contains(ClientFaultCause.ServerRetryable); - } - - return false; - } - - // This method wraps some client specific exceptions into specific ClientException or just ClientException - // ClientException will be also wrapped - public RuntimeException wrapException(String message, Exception cause, String queryId) { - if (cause instanceof ClientException || cause instanceof ServerException) { - return (RuntimeException) cause; - } - - if (cause instanceof ConnectionRequestTimeoutException || - cause instanceof NoHttpResponseException || - cause instanceof ConnectTimeoutException || - cause instanceof ConnectException || - cause instanceof UnknownHostException || - cause instanceof NoRouteToHostException) { - ConnectionInitiationException ex = new ConnectionInitiationException(message, cause); - ex.setQueryId(queryId); - return ex; - } - - if (cause instanceof SocketTimeoutException || cause instanceof IOException) { - DataTransferException ex = new DataTransferException(message, cause); - ex.setQueryId(queryId); - return ex; - } - // if we can not identify the exception explicitly we catch as our base exception ClickHouseException - return new ClickHouseException(message, cause, queryId); - } - - private void correctUserAgentHeader(HttpRequest request, Map requestConfig) { - //TODO: implement cache for user-agent - Header userAgentHeader = request.getLastHeader(HttpHeaders.USER_AGENT); - request.removeHeaders(HttpHeaders.USER_AGENT); - - String clientName = ClientConfigProperties.CLIENT_NAME.getOrDefault(requestConfig); - String userAgentValue = defaultUserAgent; - if (userAgentHeader == null && clientName != null && !clientName.isEmpty()) { - userAgentValue = clientName + " " + defaultUserAgent; - } else if (userAgentHeader != null) { - userAgentValue = userAgentHeader.getValue() + " " + defaultUserAgent; - } - request.setHeader(HttpHeaders.USER_AGENT, userAgentValue); - } - - private String buildDefaultUserAgent() { - StringBuilder userAgent = new StringBuilder(); - userAgent.append(Client.CLIENT_USER_AGENT); - - String clientVersion = Client.clientVersion; - - userAgent.append(clientVersion); - - userAgent.append(" ("); - userAgent.append(System.getProperty("os.name")); - userAgent.append("; "); - userAgent.append("jvm:").append(System.getProperty("java.version")); - userAgent.append("; "); - - userAgent.setLength(userAgent.length() - 2); - userAgent.append(')'); - - try { - String httpClientVersion = this.httpClient.getClass().getPackage().getImplementationVersion(); - if (Objects.equals(this.httpClient.getClass().getPackage().getImplementationTitle(), this.getClass().getPackage().getImplementationTitle())) { - // shaded jar - all packages have same implementation title - httpClientVersion = "unknown"; - try (InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream("client-v2-version.properties")) { - Properties p = new Properties(); - p.load(in); - - String tmp = p.getProperty("apache.http.client.version"); - if (tmp != null && !tmp.isEmpty() && !tmp.equals("${apache.httpclient.version}")) { - httpClientVersion = tmp; - } - } catch (Exception e) { - // ignore - } - } - userAgent.append(" ") - .append("Apache-HttpClient") - .append('/') - .append(httpClientVersion); - } catch (Exception e) { - LOG.info("failed to construct http client version string"); - } - return userAgent.toString(); - } - - public void close() { - httpClient.close(CloseMode.IMMEDIATE); - } - - private static void setHeader(HttpRequest req, String headerName, - String value) - { - if (value == null) { - return; - } - - if (value.trim().isEmpty()) { - return; - } - if (PATTERN_HEADER_VALUE_ASCII.matcher(value).matches()) { - req.setHeader(headerName, value); - } else { - try { - req.setHeader( - headerName + "*", - "UTF-8''" + URLEncoder.encode(value, StandardCharsets.UTF_8.name())); - } catch (UnsupportedEncodingException e) { - throw new ClientException("Failed to convert string to UTF8" , e); - } - } - } - - /** - * This factory is used only when no ssl connections are required (no https endpoints). - * Internally http client would create factory and spend time if no supplied. - */ - private static class DummySSLConnectionSocketFactory implements LayeredConnectionSocketFactory { - @Override - public Socket createLayeredSocket(Socket socket, String target, int port, HttpContext context) throws IOException { - return null; - } - - @Override - public Socket createSocket(HttpContext context) throws IOException { - return null; - } - - @Override - public Socket connectSocket(TimeValue connectTimeout, Socket socket, HttpHost host, InetSocketAddress remoteAddress, InetSocketAddress localAddress, HttpContext context) throws IOException { - return null; - } - } - - public static class MeteredManagedHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory { - public MeteredManagedHttpClientConnectionFactory(Http1Config http1Config, CharCodingConfig charCodingConfig, DefaultHttpResponseParserFactory defaultHttpResponseParserFactory) { - super(http1Config, charCodingConfig, defaultHttpResponseParserFactory); - } - - ConcurrentLinkedQueue times = new ConcurrentLinkedQueue<>(); - - - @Override - public ManagedHttpClientConnection createConnection(Socket socket) throws IOException { - long startT = System.currentTimeMillis(); - try { - return super.createConnection(socket); - } finally { - long endT = System.currentTimeMillis(); - times.add(endT - startT); - } - } - - public long getTime() { - int count = times.size(); - long runningAverage = 0; - for (int i = 0; i < count; i++) { - Long t = times.poll(); - if (t != null) { - runningAverage += t; - } - } - - return count > 0 ? runningAverage / count : 0; - } - } - - public static class CustomSSLConnectionFactory extends SSLConnectionSocketFactory { - - private final SNIHostName defaultSNI; - - public CustomSSLConnectionFactory(String defaultSNI, SSLContext sslContext, HostnameVerifier hostnameVerifier) { - super(sslContext, hostnameVerifier); - this.defaultSNI = defaultSNI == null || defaultSNI.trim().isEmpty() ? null : new SNIHostName(defaultSNI); - } - - @Override - protected void prepareSocket(SSLSocket socket, HttpContext context) throws IOException { - super.prepareSocket(socket, context); - - if (defaultSNI != null) { - SSLParameters sslParams = socket.getSSLParameters(); - sslParams.setServerNames(Collections.singletonList(defaultSNI)); - socket.setSSLParameters(sslParams); - } - } - } -} +package com.clickhouse.client.api.internal; + +import com.clickhouse.client.ClickHouseSslContextProvider; +import com.clickhouse.client.api.ClickHouseException; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.ClientFaultCause; +import com.clickhouse.client.api.ClientMisconfigurationException; +import com.clickhouse.client.api.ConnectionInitiationException; +import com.clickhouse.client.api.ConnectionReuseStrategy; +import com.clickhouse.client.api.DataTransferException; +import com.clickhouse.client.api.ServerException; +import com.clickhouse.client.api.enums.ProxyType; +import com.clickhouse.client.api.http.ClickHouseHttpProto; +import com.clickhouse.client.api.transport.Endpoint; +import com.clickhouse.data.ClickHouseFormat; +import net.jpountz.lz4.LZ4Factory; +import org.apache.commons.compress.compressors.CompressorStreamFactory; +import org.apache.hc.client5.http.ConnectTimeoutException; +import org.apache.hc.client5.http.classic.methods.HttpPost; +import org.apache.hc.client5.http.config.ConnectionConfig; +import org.apache.hc.client5.http.config.RequestConfig; +import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.impl.io.BasicHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.client5.http.io.ManagedHttpClientConnection; +import org.apache.hc.client5.http.protocol.HttpClientContext; +import org.apache.hc.client5.http.socket.ConnectionSocketFactory; +import org.apache.hc.client5.http.socket.LayeredConnectionSocketFactory; +import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory; +import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; +import org.apache.hc.core5.http.ClassicHttpResponse; +import org.apache.hc.core5.http.ConnectionRequestTimeoutException; +import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.HttpEntity; +import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.HttpHost; +import org.apache.hc.core5.http.HttpRequest; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.http.NoHttpResponseException; +import org.apache.hc.core5.http.config.CharCodingConfig; +import org.apache.hc.core5.http.config.Http1Config; +import org.apache.hc.core5.http.config.RegistryBuilder; +import org.apache.hc.core5.http.impl.io.DefaultHttpResponseParserFactory; +import org.apache.hc.core5.http.io.SocketConfig; +import org.apache.hc.core5.http.io.entity.ByteArrayEntity; +import org.apache.hc.core5.http.io.entity.EntityTemplate; +import org.apache.hc.core5.http.protocol.HttpContext; +import org.apache.hc.core5.io.CloseMode; +import org.apache.hc.core5.io.IOCallback; +import org.apache.hc.core5.net.URIBuilder; +import org.apache.hc.core5.pool.ConnPoolControl; +import org.apache.hc.core5.pool.PoolConcurrencyPolicy; +import org.apache.hc.core5.pool.PoolReusePolicy; +import org.apache.hc.core5.util.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SNIHostName; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLException; +import javax.net.ssl.SSLParameters; +import javax.net.ssl.SSLSocket; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.Method; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; +import java.net.Socket; +import java.net.SocketTimeoutException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.net.UnknownHostException; +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Base64; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +public class HttpAPIClientHelper { + + public static final String KEY_STATEMENT_PARAMS = "statement_params"; + + private static final Logger LOG = LoggerFactory.getLogger(HttpAPIClientHelper.class); + + private static final int ERROR_BODY_BUFFER_SIZE = 1024; // Error messages are usually small + + private final String DEFAULT_HTTP_COMPRESSION_ALGO = "lz4"; + + private static final Pattern PATTERN_HEADER_VALUE_ASCII = Pattern.compile( + "\\p{Graph}+(?:[ ]\\p{Graph}+)*"); + + private final CloseableHttpClient httpClient; + + private String proxyAuthHeaderValue; + + private final Set defaultRetryCauses; + + private final String defaultUserAgent; + private final Object metricsRegistry; + + ConnPoolControl poolControl; + + LZ4Factory lz4Factory; + + public HttpAPIClientHelper(Map configuration, Object metricsRegistry, boolean initSslContext, LZ4Factory lz4Factory) { + this.metricsRegistry = metricsRegistry; + this.httpClient = createHttpClient(initSslContext, configuration); + this.lz4Factory = lz4Factory; + assert this.lz4Factory != null; + + boolean usingClientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration); + boolean usingServerCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(configuration); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(configuration); + + LOG.debug("client compression: {}, server compression: {}, http compression: {}", usingClientCompression, usingServerCompression, useHttpCompression); + + defaultRetryCauses = new HashSet<>(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(configuration)); + if (defaultRetryCauses.contains(ClientFaultCause.None)) { + defaultRetryCauses.removeIf(c -> c != ClientFaultCause.None); + } + + this.defaultUserAgent = buildDefaultUserAgent(); + } + + /** + * Creates or returns default SSL context. + * + * @return SSLContext + */ + public SSLContext createSSLContext(Map configuration) { + SSLContext sslContext; + try { + sslContext = SSLContext.getDefault(); + } catch (NoSuchAlgorithmException e) { + throw new ClientException("Failed to create default SSL context", e); + } + ClickHouseSslContextProvider sslContextProvider = ClickHouseSslContextProvider.getProvider(); + String trustStorePath = (String) configuration.get(ClientConfigProperties.SSL_TRUST_STORE.getKey()); + if (trustStorePath != null) { + try { + sslContext = sslContextProvider.getSslContextFromKeyStore( + trustStorePath, + (String) configuration.get(ClientConfigProperties.SSL_KEY_STORE_PASSWORD.getKey()), + (String) configuration.get(ClientConfigProperties.SSL_KEYSTORE_TYPE.getKey()) + ); + } catch (SSLException e) { + throw new ClientMisconfigurationException("Failed to create SSL context from a keystore", e); + } + } else if (configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) != null || + configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()) != null || + configuration.get(ClientConfigProperties.SSL_KEY.getKey()) != null) { + + try { + sslContext = sslContextProvider.getSslContextFromCerts( + (String) configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()), + (String) configuration.get(ClientConfigProperties.SSL_KEY.getKey()), + (String) configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) + ); + } catch (SSLException e) { + throw new ClientMisconfigurationException("Failed to create SSL context from certificates", e); + } + } + return sslContext; + } + + private static final long CONNECTION_INACTIVITY_CHECK = 5000L; + + private ConnectionConfig createConnectionConfig(Map configuration) { + ConnectionConfig.Builder connConfig = ConnectionConfig.custom(); + + ClientConfigProperties.CONNECTION_TTL.applyIfSet(configuration, (t) -> connConfig.setTimeToLive(t, TimeUnit.MILLISECONDS)); + ClientConfigProperties.CONNECTION_TIMEOUT.applyIfSet(configuration, (t) -> connConfig.setConnectTimeout(t, TimeUnit.MILLISECONDS)); + connConfig.setValidateAfterInactivity(CONNECTION_INACTIVITY_CHECK, TimeUnit.MILLISECONDS); // non-configurable for now + + return connConfig.build(); + } + + private HttpClientConnectionManager basicConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { + RegistryBuilder registryBuilder = RegistryBuilder.create(); + registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory()); + registryBuilder.register("https", sslConnectionSocketFactory); + + BasicHttpClientConnectionManager connManager = new BasicHttpClientConnectionManager(registryBuilder.build()); + connManager.setConnectionConfig(createConnectionConfig(configuration)); + connManager.setSocketConfig(socketConfig); + + return connManager; + } + + private HttpClientConnectionManager poolConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { + PoolingHttpClientConnectionManagerBuilder connMgrBuilder = PoolingHttpClientConnectionManagerBuilder.create() + .setPoolConcurrencyPolicy(PoolConcurrencyPolicy.LAX); + + ConnectionReuseStrategy connectionReuseStrategy = ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getOrDefault(configuration); + switch (connectionReuseStrategy) { + case LIFO: + connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.LIFO); + break; + case FIFO: + connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.FIFO); + break; + default: + throw new ClientMisconfigurationException("Unknown connection reuse strategy: " + connectionReuseStrategy); + } + LOG.debug("Connection reuse strategy: {}", connectionReuseStrategy); + + connMgrBuilder.setDefaultConnectionConfig(createConnectionConfig(configuration)); + connMgrBuilder.setMaxConnTotal(Integer.MAX_VALUE); // as we do not know how many routes we will have + ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.applyIfSet(configuration, connMgrBuilder::setMaxConnPerRoute); + + int networkBufferSize = ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getOrDefault(configuration); + MeteredManagedHttpClientConnectionFactory connectionFactory = new MeteredManagedHttpClientConnectionFactory( + Http1Config.custom() + .setBufferSize(networkBufferSize) + .build(), + CharCodingConfig.DEFAULT, + DefaultHttpResponseParserFactory.INSTANCE); + + connMgrBuilder.setConnectionFactory(connectionFactory); + connMgrBuilder.setSSLSocketFactory(sslConnectionSocketFactory); + connMgrBuilder.setDefaultSocketConfig(socketConfig); + PoolingHttpClientConnectionManager phccm = connMgrBuilder.build(); + poolControl = phccm; + if (metricsRegistry != null) { + try { + String mGroupName = ClientConfigProperties.METRICS_GROUP_NAME.getOrDefault(configuration); + Class micrometerLoader = getClass().getClassLoader().loadClass("com.clickhouse.client.api.metrics.MicrometerLoader"); + Method applyMethod = micrometerLoader.getDeclaredMethod("applyPoolingMetricsBinder", Object.class, String.class, PoolingHttpClientConnectionManager.class); + applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, phccm); + + applyMethod = micrometerLoader.getDeclaredMethod("applyConnectionMetricsBinder", Object.class, String.class, MeteredManagedHttpClientConnectionFactory.class); + applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, connectionFactory); + } catch (Exception e) { + LOG.error("Failed to register metrics", e); + } + } + return phccm; + } + + public CloseableHttpClient createHttpClient(boolean initSslContext, Map configuration) { + // Top Level builders + HttpClientBuilder clientBuilder = HttpClientBuilder.create(); + SSLContext sslContext = initSslContext ? createSSLContext(configuration) : null; + LayeredConnectionSocketFactory sslConnectionSocketFactory; + if (sslContext != null) { + String socketSNI = (String)configuration.get(ClientConfigProperties.SSL_SOCKET_SNI.getKey()); + if (socketSNI != null && !socketSNI.trim().isEmpty()) { + sslConnectionSocketFactory = new CustomSSLConnectionFactory(socketSNI, sslContext, (hostname, session) -> true); + } else { + sslConnectionSocketFactory = new SSLConnectionSocketFactory(sslContext); + } + } else { + sslConnectionSocketFactory = new DummySSLConnectionSocketFactory(); + } + // Socket configuration + SocketConfig.Builder soCfgBuilder = SocketConfig.custom(); + ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.applyIfSet(configuration, + (t) -> soCfgBuilder.setSoTimeout(t, TimeUnit.MILLISECONDS)); + + ClientConfigProperties.SOCKET_RCVBUF_OPT.applyIfSet(configuration, + soCfgBuilder::setRcvBufSize); + + ClientConfigProperties.SOCKET_SNDBUF_OPT.applyIfSet(configuration, + soCfgBuilder::setSndBufSize); + + ClientConfigProperties.SOCKET_LINGER_OPT.applyIfSet(configuration, + (v) -> soCfgBuilder.setSoLinger(v, TimeUnit.SECONDS)); + + ClientConfigProperties.SOCKET_TCP_NO_DELAY_OPT.applyIfSet(configuration, + soCfgBuilder::setTcpNoDelay); + // Proxy + String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey()); + Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey()); + HttpHost proxy = null; + if (proxyHost != null && proxyPort != null) { + proxy = new HttpHost(proxyHost, proxyPort); + } + + String proxyTypeVal = (String) configuration.get(ClientConfigProperties.PROXY_TYPE.getKey()); + ProxyType proxyType = proxyTypeVal == null ? null : ProxyType.valueOf(proxyTypeVal); + if (proxyType == ProxyType.HTTP) { + clientBuilder.setProxy(proxy); + String proxyUser = (String) configuration.get(ClientConfigProperties.PROXY_USER.getKey()); + String proxyPassword = (String) configuration.get(ClientConfigProperties.PROXY_PASSWORD.getKey()); + if (proxyUser != null && proxyPassword != null) { + proxyAuthHeaderValue = "Basic " + Base64.getEncoder().encodeToString((proxyUser + ":" + proxyPassword).getBytes(StandardCharsets.UTF_8)); + } + + } else if (proxyType == ProxyType.SOCKS) { + soCfgBuilder.setSocksProxyAddress(new InetSocketAddress(proxyHost, proxyPort)); + } + + boolean disableCookies = !((Boolean)ClientConfigProperties.HTTP_SAVE_COOKIES.getOrDefault(configuration)); + if (disableCookies) { + clientBuilder.disableCookieManagement(); + } + SocketConfig socketConfig = soCfgBuilder.build(); + + // Connection manager + if (ClientConfigProperties.CONNECTION_POOL_ENABLED.getOrDefault(configuration)) { + clientBuilder.setConnectionManager(poolConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); + } else { + clientBuilder.setConnectionManager(basicConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); + } + Long keepAliveTimeout = ClientConfigProperties.HTTP_KEEP_ALIVE_TIMEOUT.getOrDefault(configuration); + if (keepAliveTimeout != null && keepAliveTimeout > 0) { + clientBuilder.setKeepAliveStrategy((response, context) -> TimeValue.ofMilliseconds(keepAliveTimeout)); + } + + clientBuilder.disableContentCompression(); // will handle ourselves + + return clientBuilder.build(); + } + + private static final String ERROR_CODE_PREFIX_PATTERN = "%d. DB::Exception:"; + + /** + * Reads status line and if error tries to parse response body to get server error message. + * + * @param httpResponse - HTTP response + * @return exception object with server code + */ + public Exception readError(HttpPost req, ClassicHttpResponse httpResponse) { + final Header serverQueryIdHeader = httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); + final Header clientQueryIdHeader = req.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); + final Header queryHeader = Stream.of(serverQueryIdHeader, clientQueryIdHeader).filter(Objects::nonNull).findFirst().orElse(null); + final String queryId = queryHeader == null ? "" : queryHeader.getValue(); + int serverCode = getHeaderInt(httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE), 0); + try { + return serverCode > 0 ? readClickHouseError(httpResponse.getEntity(), serverCode, queryId, httpResponse.getCode()) : + readNotClickHouseError(httpResponse.getEntity(), queryId, httpResponse.getCode()); + } catch (Exception e) { + LOG.error("Failed to read error message", e); + String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; + return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); + } + } + + private ServerException readNotClickHouseError(HttpEntity httpEntity, String queryId, int httpCode) { + + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + + String msg = null; + InputStream body = null; + for (int i = 0; i < 2; i++) { + try { + if (body == null) { + body = httpEntity.getContent(); + } + int msgLen = body.read(buffer, 0, buffer.length - 2); + if (msgLen > 0) { + msg = new String(buffer, 0, msgLen, StandardCharsets.UTF_8).trim(); + if (msg.isEmpty()) { + msg = ""; + } + } + break; + } catch (ClientException e) { + // Invalid LZ4 Magic + if (body instanceof ClickHouseLZ4InputStream) { + ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; + body = stream.getInputStream(); + continue; + } + throw e; + } catch (Exception e) { + LOG.warn("Failed to read error message (queryId = " + queryId + ")", e); + break; + } + } + + String errormsg = msg == null ? "unknown server error" : msg; + return new ServerException(ServerException.CODE_UNKNOWN, errormsg + " (transport error: " + httpCode +")", httpCode, queryId); + } + + private static ServerException readClickHouseError(HttpEntity httpEntity, int serverCode, String queryId, int httpCode) throws Exception { + InputStream body = httpEntity.getContent(); + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + byte[] lookUpStr = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode).getBytes(StandardCharsets.UTF_8); + StringBuilder msgBuilder = new StringBuilder(); + boolean found = false; + while (true) { + int rBytes = -1; + try { + rBytes = body.read(buffer); + } catch (ClientException e) { + // Invalid LZ4 Magic + if (body instanceof ClickHouseLZ4InputStream) { + ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; + body = stream.getInputStream(); + byte[] headerBuffer = stream.getHeaderBuffer(); + System.arraycopy(headerBuffer, 0, buffer, 0, headerBuffer.length); + rBytes = headerBuffer.length; + } + } + if (rBytes == -1) { + break; + } + + for (int i = 0; i < rBytes; i++) { + if (buffer[i] == lookUpStr[0]) { + found = true; + for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { + if (buffer[i + j] != lookUpStr[j]) { + found = false; + break; + } + } + if (found) { + msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); + break; + } + } + } + + if (found) { + break; + } + } + + while (true) { + int rBytes = body.read(buffer); + if (rBytes == -1) { + break; + } + msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); + } + + String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") + .replaceAll("\\\\/", "/"); + if (msg.trim().isEmpty()) { + msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpCode + ")"; + } + return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpCode, queryId); + } + + private static final long POOL_VENT_TIMEOUT = 10000L; + private final AtomicLong timeToPoolVent = new AtomicLong(0); + + private void doPoolVent() { + if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) { + timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT); + poolControl.closeExpired(); + } + } + + private HttpContext createRequestHttpContext(Map requestConfig) { + HttpClientContext context = HttpClientContext.create(); + Number responseTimeout = ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getOrDefault(requestConfig); + Number connectionReqTimeout = ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getOrDefault(requestConfig); + RequestConfig reqHttpConf = RequestConfig.custom() + .setResponseTimeout(responseTimeout.longValue(), TimeUnit.MILLISECONDS) + .setConnectionRequestTimeout(connectionReqTimeout.longValue(), TimeUnit.MILLISECONDS) + .build(); + context.setRequestConfig(reqHttpConf); + + return context; + } + + private URI createRequestURI(Endpoint server, Map requestConfig, boolean addParameters) { + URI uri; + try { + URIBuilder uriBuilder = new URIBuilder(server.getURI()); + addRequestParams(requestConfig, uriBuilder::addParameter); + + if (addParameters) { + addStatementParams(requestConfig, uriBuilder::addParameter); + } + + uri = uriBuilder.optimize().build(); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + return uri; + } + + private HttpPost createPostRequest(URI uri, Map requestConfig) { + HttpPost req = new HttpPost(uri); +// req.setVersion(new ProtocolVersion("HTTP", 1, 0)); // to disable chunk transfer encoding + addHeaders(req, requestConfig); + return req; + } + + public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, + String body) throws Exception { + + final URI uri = createRequestURI(server, requestConfig, true); + final HttpPost req = createPostRequest(uri, requestConfig); + final String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; + + HttpEntity httpEntity = new ByteArrayEntity(body.getBytes(StandardCharsets.UTF_8.name()), CONTENT_TYPE, contentEncoding); + req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); + + return doPostRequest(requestConfig, req); + } + + public ClassicHttpResponse executeMultiPartRequest(Endpoint server, Map requestConfig, String sqlQuery) throws Exception { + + requestConfig.put(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey(), false); + + final URI uri = createRequestURI(server, requestConfig, false); + final HttpPost req = createPostRequest(uri, requestConfig); + + MultipartEntityBuilder multipartEntityBuilder = MultipartEntityBuilder.create(); + addStatementParams(requestConfig, multipartEntityBuilder::addTextBody); + multipartEntityBuilder.addTextBody(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlQuery); + + + HttpEntity httpEntity = multipartEntityBuilder.build(); + req.setHeader(HttpHeaders.CONTENT_TYPE, httpEntity.getContentType()); // set proper content type with generated boundary value + req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); + + return doPostRequest(requestConfig, req); + + + } + + public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, + IOCallback writeCallback) throws Exception { + + final URI uri = createRequestURI(server, requestConfig, true); + final HttpPost req = createPostRequest(uri, requestConfig); + String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; + req.setEntity(wrapRequestEntity( + new EntityTemplate(-1, CONTENT_TYPE, contentEncoding , writeCallback), + requestConfig)); + + return doPostRequest(requestConfig, req); + } + + private ClassicHttpResponse doPostRequest(Map requestConfig, HttpPost req) throws Exception { + + doPoolVent(); + + ClassicHttpResponse httpResponse = null; + HttpContext context = createRequestHttpContext(requestConfig); + try { + httpResponse = httpClient.executeOpen(null, req, context); + + httpResponse.setEntity(wrapResponseEntity(httpResponse.getEntity(), + httpResponse.getCode(), + requestConfig)); + + if (httpResponse.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) { + throw new ClientMisconfigurationException("Proxy authentication required. Please check your proxy settings."); + } else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) { + httpResponse.close(); + throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings."); + } else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) { + try { + throw readError(req, httpResponse); + } finally { + httpResponse.close(); + } + } + return httpResponse; + + } catch (UnknownHostException e) { + closeQuietly(httpResponse); + LOG.warn("Host '{}' unknown", req.getAuthority()); + throw e; + } catch (ConnectException | NoRouteToHostException e) { + closeQuietly(httpResponse); + LOG.warn("Failed to connect to '{}': {}", req.getAuthority(), e.getMessage()); + throw e; + } catch (Exception e) { + closeQuietly(httpResponse); + LOG.debug("Failed to execute request to '{}': {}", req.getAuthority(), e.getMessage(), e); + throw e; + } + } + + public static void closeQuietly(ClassicHttpResponse httpResponse) { + if (httpResponse != null) { + try { + httpResponse.close(); + } catch (IOException e) { + LOG.warn("Failed to close response"); + } + } + } + + private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8"); + + private void addHeaders(HttpPost req, Map requestConfig) { + setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType()); + if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) { + setHeader( + req, + ClickHouseHttpProto.HEADER_FORMAT, + ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name()); + } + if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { + setHeader( + req, + ClickHouseHttpProto.HEADER_QUERY_ID, + (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey())); + } + setHeader( + req, + ClickHouseHttpProto.HEADER_DATABASE, + ClientConfigProperties.DATABASE.getOrDefault(requestConfig)); + + if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) { + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_USER, + ClientConfigProperties.USER.getOrDefault(requestConfig)); + setHeader( + req, + ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, + "on"); + } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) { + String user = ClientConfigProperties.USER.getOrDefault(requestConfig); + String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig); + if (password == null) { + password = ""; + } + // Use as-is, no encoding allowed + req.addHeader( + HttpHeaders.AUTHORIZATION, + "Basic " + Base64.getEncoder().encodeToString( + (user + ":" + password).getBytes(StandardCharsets.UTF_8))); + } else { + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_USER, + ClientConfigProperties.USER.getOrDefault(requestConfig)); + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_PASSWORD, + ClientConfigProperties.PASSWORD.getOrDefault(requestConfig)); + } + if (proxyAuthHeaderValue != null) { + req.addHeader( + HttpHeaders.PROXY_AUTHORIZATION, + proxyAuthHeaderValue); + } + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); + + if (useHttpCompression) { + if (serverCompression) { + setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); + } + + if (clientCompression && !appCompressedData) { + setHeader(req, HttpHeaders.CONTENT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); + } + } + + for (String key : requestConfig.keySet()) { + if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { + Object val = requestConfig.get(key); + if (val != null) { + setHeader( + req, + key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), + String.valueOf(val)); + } + } + } + + // Special cases + if (req.containsHeader(HttpHeaders.AUTHORIZATION) + && (req.containsHeader(ClickHouseHttpProto.HEADER_DB_USER) || + req.containsHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD))) + { + // user has set auth header for purpose, lets remove ours + req.removeHeaders(ClickHouseHttpProto.HEADER_DB_USER); + req.removeHeaders(ClickHouseHttpProto.HEADER_DB_PASSWORD); + } + + // -- keep last + correctUserAgentHeader(req, requestConfig); + } + + private void addRequestParams(Map requestConfig, BiConsumer consumer) { + if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { + consumer.accept(ClickHouseHttpProto.QPARAM_QUERY_ID, requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()).toString()); + } + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + + if (useHttpCompression) { + // enable_http_compression make server react on http header + // for client side compression Content-Encoding should be set + // for server side compression Accept-Encoding should be set + consumer.accept("enable_http_compression", "1"); + } else { + if (serverCompression) { + consumer.accept("compress", "1"); + } + if (clientCompression) { + consumer.accept("decompress", "1"); + } + } + + Collection sessionRoles = ClientConfigProperties.SESSION_DB_ROLES.getOrDefault(requestConfig); + if (!(sessionRoles == null || sessionRoles.isEmpty())) { + sessionRoles.forEach(r -> consumer.accept(ClickHouseHttpProto.QPARAM_ROLE, r)); + } + + for (String key : requestConfig.keySet()) { + if (key.startsWith(ClientConfigProperties.SERVER_SETTING_PREFIX)) { + Object val = requestConfig.get(key); + if (val != null) { + consumer.accept(key.substring(ClientConfigProperties.SERVER_SETTING_PREFIX.length()), String.valueOf(requestConfig.get(key))); + } + } + } + } + + private void addStatementParams(Map requestConfig, BiConsumer consumer) { + if (requestConfig.containsKey(KEY_STATEMENT_PARAMS)) { + Map params = (Map) requestConfig.get(KEY_STATEMENT_PARAMS); + params.forEach((k, v) -> consumer.accept("param_" + k, String.valueOf(v))); + } + } + + private HttpEntity wrapRequestEntity(HttpEntity httpEntity, Map requestConfig) { + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); + + LOG.debug("wrapRequestEntity: client compression: {}, http compression: {}, content encoding: {}", + clientCompression, useHttpCompression, httpEntity.getContentEncoding()); + + if (httpEntity.getContentEncoding() != null && !appCompressedData) { + // http header is set and data is not compressed + return new CompressedEntity(httpEntity, false, CompressorStreamFactory.getSingleton()); + } else if (clientCompression && !appCompressedData) { + int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); + return new LZ4Entity(httpEntity, useHttpCompression, false, true, + buffSize, false, lz4Factory); + } else { + return httpEntity; + } + } + + private HttpEntity wrapResponseEntity(HttpEntity httpEntity, int httpStatus, Map requestConfig) { + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + + LOG.debug("wrapResponseEntity: server compression: {}, http compression: {}, content encoding: {}", + serverCompression, useHttpCompression, httpEntity.getContentEncoding()); + + if (httpEntity.getContentEncoding() != null) { + // http compressed response + return new CompressedEntity(httpEntity, true, CompressorStreamFactory.getSingleton()); + } + + // data compression + if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED || httpStatus == HttpStatus.SC_NOT_FOUND)) { + int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); + return new LZ4Entity(httpEntity, useHttpCompression, true, false, buffSize, true, lz4Factory); + } + + return httpEntity; + } + + public static int getHeaderInt(Header header, int defaultValue) { + return getHeaderVal(header, defaultValue, Integer::parseInt); + } + + private static final Set RESPONSE_HEADER_WHITELIST = new HashSet<>(Arrays.asList( + ClickHouseHttpProto.HEADER_QUERY_ID, + ClickHouseHttpProto.HEADER_SRV_SUMMARY, + ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME, + ClickHouseHttpProto.HEADER_DATABASE, + ClickHouseHttpProto.HEADER_DB_USER + )); + + /** + * Collects whitelisted response headers from an HTTP response into a map. + * + * @param response the HTTP response + * @return unmodifiable map of header name to header value for whitelisted headers present in the response + */ + public static Map collectResponseHeaders(ClassicHttpResponse response) { + Map headers = new HashMap<>(); + for (String name : RESPONSE_HEADER_WHITELIST) { + Header header = response.getFirstHeader(name); + if (header != null) { + headers.put(name, header.getValue()); + } + } + return Collections.unmodifiableMap(headers); + } + + public static String getHeaderVal(Header header, String defaultValue) { + return getHeaderVal(header, defaultValue, Function.identity()); + } + + public static T getHeaderVal(Header header, T defaultValue, Function converter) { + if (header == null) { + return defaultValue; + } + + return converter.apply(header.getValue()); + } + + public boolean shouldRetry(Throwable ex, Map requestSettings) { + List retryCauses = ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(requestSettings); + + if (retryCauses.contains(ClientFaultCause.None)) { + return false; + } + + if (ex instanceof NoHttpResponseException + || ex.getCause() instanceof NoHttpResponseException) { + return retryCauses.contains(ClientFaultCause.NoHttpResponse); + } + + if (ex instanceof ConnectException + || ex instanceof ConnectTimeoutException + || ex.getCause() instanceof ConnectException + || ex.getCause() instanceof ConnectTimeoutException) { + return retryCauses.contains(ClientFaultCause.ConnectTimeout); + } + + if (ex instanceof ConnectionRequestTimeoutException + || ex.getCause() instanceof ConnectionRequestTimeoutException) { + return retryCauses.contains(ClientFaultCause.ConnectionRequestTimeout); + } + + if (ex instanceof SocketTimeoutException + || ex.getCause() instanceof SocketTimeoutException) { + return retryCauses.contains(ClientFaultCause.SocketTimeout); + } + + // there are some db retryable error codes + if (ex instanceof ServerException || ex.getCause() instanceof ServerException) { + ServerException se = (ServerException) ex; + return se.isRetryable() && retryCauses.contains(ClientFaultCause.ServerRetryable); + } + + return false; + } + + // This method wraps some client specific exceptions into specific ClientException or just ClientException + // ClientException will be also wrapped + public RuntimeException wrapException(String message, Exception cause, String queryId) { + if (cause instanceof ClientException || cause instanceof ServerException) { + return (RuntimeException) cause; + } + + if (cause instanceof ConnectionRequestTimeoutException || + cause instanceof NoHttpResponseException || + cause instanceof ConnectTimeoutException || + cause instanceof ConnectException || + cause instanceof UnknownHostException || + cause instanceof NoRouteToHostException) { + ConnectionInitiationException ex = new ConnectionInitiationException(message, cause); + ex.setQueryId(queryId); + return ex; + } + + if (cause instanceof SocketTimeoutException || cause instanceof IOException) { + DataTransferException ex = new DataTransferException(message, cause); + ex.setQueryId(queryId); + return ex; + } + // if we can not identify the exception explicitly we catch as our base exception ClickHouseException + return new ClickHouseException(message, cause, queryId); + } + + private void correctUserAgentHeader(HttpRequest request, Map requestConfig) { + //TODO: implement cache for user-agent + Header userAgentHeader = request.getLastHeader(HttpHeaders.USER_AGENT); + request.removeHeaders(HttpHeaders.USER_AGENT); + + String clientName = ClientConfigProperties.CLIENT_NAME.getOrDefault(requestConfig); + String userAgentValue = defaultUserAgent; + if (userAgentHeader == null && clientName != null && !clientName.isEmpty()) { + userAgentValue = clientName + " " + defaultUserAgent; + } else if (userAgentHeader != null) { + userAgentValue = userAgentHeader.getValue() + " " + defaultUserAgent; + } + request.setHeader(HttpHeaders.USER_AGENT, userAgentValue); + } + + private String buildDefaultUserAgent() { + StringBuilder userAgent = new StringBuilder(); + userAgent.append(Client.CLIENT_USER_AGENT); + + String clientVersion = Client.clientVersion; + + userAgent.append(clientVersion); + + userAgent.append(" ("); + userAgent.append(System.getProperty("os.name")); + userAgent.append("; "); + userAgent.append("jvm:").append(System.getProperty("java.version")); + userAgent.append("; "); + + userAgent.setLength(userAgent.length() - 2); + userAgent.append(')'); + + try { + String httpClientVersion = this.httpClient.getClass().getPackage().getImplementationVersion(); + if (Objects.equals(this.httpClient.getClass().getPackage().getImplementationTitle(), this.getClass().getPackage().getImplementationTitle())) { + // shaded jar - all packages have same implementation title + httpClientVersion = "unknown"; + try (InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream("client-v2-version.properties")) { + Properties p = new Properties(); + p.load(in); + + String tmp = p.getProperty("apache.http.client.version"); + if (tmp != null && !tmp.isEmpty() && !tmp.equals("${apache.httpclient.version}")) { + httpClientVersion = tmp; + } + } catch (Exception e) { + // ignore + } + } + userAgent.append(" ") + .append("Apache-HttpClient") + .append('/') + .append(httpClientVersion); + } catch (Exception e) { + LOG.info("failed to construct http client version string"); + } + return userAgent.toString(); + } + + public void close() { + httpClient.close(CloseMode.IMMEDIATE); + } + + private static void setHeader(HttpRequest req, String headerName, + String value) + { + if (value == null) { + return; + } + + if (value.trim().isEmpty()) { + return; + } + if (PATTERN_HEADER_VALUE_ASCII.matcher(value).matches()) { + req.setHeader(headerName, value); + } else { + try { + req.setHeader( + headerName + "*", + "UTF-8''" + URLEncoder.encode(value, StandardCharsets.UTF_8.name())); + } catch (UnsupportedEncodingException e) { + throw new ClientException("Failed to convert string to UTF8" , e); + } + } + } + + /** + * This factory is used only when no ssl connections are required (no https endpoints). + * Internally http client would create factory and spend time if no supplied. + */ + private static class DummySSLConnectionSocketFactory implements LayeredConnectionSocketFactory { + @Override + public Socket createLayeredSocket(Socket socket, String target, int port, HttpContext context) throws IOException { + return null; + } + + @Override + public Socket createSocket(HttpContext context) throws IOException { + return null; + } + + @Override + public Socket connectSocket(TimeValue connectTimeout, Socket socket, HttpHost host, InetSocketAddress remoteAddress, InetSocketAddress localAddress, HttpContext context) throws IOException { + return null; + } + } + + public static class MeteredManagedHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory { + public MeteredManagedHttpClientConnectionFactory(Http1Config http1Config, CharCodingConfig charCodingConfig, DefaultHttpResponseParserFactory defaultHttpResponseParserFactory) { + super(http1Config, charCodingConfig, defaultHttpResponseParserFactory); + } + + ConcurrentLinkedQueue times = new ConcurrentLinkedQueue<>(); + + + @Override + public ManagedHttpClientConnection createConnection(Socket socket) throws IOException { + long startT = System.currentTimeMillis(); + try { + return super.createConnection(socket); + } finally { + long endT = System.currentTimeMillis(); + times.add(endT - startT); + } + } + + public long getTime() { + int count = times.size(); + long runningAverage = 0; + for (int i = 0; i < count; i++) { + Long t = times.poll(); + if (t != null) { + runningAverage += t; + } + } + + return count > 0 ? runningAverage / count : 0; + } + } + + public static class CustomSSLConnectionFactory extends SSLConnectionSocketFactory { + + private final SNIHostName defaultSNI; + + public CustomSSLConnectionFactory(String defaultSNI, SSLContext sslContext, HostnameVerifier hostnameVerifier) { + super(sslContext, hostnameVerifier); + this.defaultSNI = defaultSNI == null || defaultSNI.trim().isEmpty() ? null : new SNIHostName(defaultSNI); + } + + @Override + protected void prepareSocket(SSLSocket socket, HttpContext context) throws IOException { + super.prepareSocket(socket, context); + + if (defaultSNI != null) { + SSLParameters sslParams = socket.getSSLParameters(); + sslParams.setServerNames(Collections.singletonList(defaultSNI)); + socket.setSSLParameters(sslParams); + } + } + } +} diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index c4bec0803..59f53594c 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -1,1737 +1,1736 @@ -package com.clickhouse.client; - -import com.clickhouse.client.api.Client; -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.ClientFaultCause; -import com.clickhouse.client.api.ConnectionInitiationException; -import com.clickhouse.client.api.ConnectionReuseStrategy; -import com.clickhouse.client.api.ServerException; -import com.clickhouse.client.api.command.CommandResponse; -import com.clickhouse.client.api.command.CommandSettings; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.enums.Protocol; -import com.clickhouse.client.api.enums.ProxyType; -import com.clickhouse.client.api.insert.InsertResponse; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.internal.ServerSettings; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.client.api.query.QueryResponse; -import com.clickhouse.client.api.query.QuerySettings; -import com.clickhouse.client.config.ClickHouseClientOption; -import com.clickhouse.data.ClickHouseFormat; -import com.github.tomakehurst.wiremock.WireMockServer; -import com.github.tomakehurst.wiremock.client.WireMock; -import com.github.tomakehurst.wiremock.common.ConsoleNotifier; -import com.github.tomakehurst.wiremock.common.Slf4jNotifier; -import com.github.tomakehurst.wiremock.core.WireMockConfiguration; -import com.github.tomakehurst.wiremock.http.Fault; -import com.github.tomakehurst.wiremock.http.trafficlistener.WiremockNetworkTrafficListener; -import org.apache.hc.core5.http.ConnectionClosedException; -import org.apache.hc.core5.http.ConnectionRequestTimeoutException; -import org.apache.hc.core5.http.HttpHeaders; -import org.apache.hc.core5.http.HttpStatus; -import org.apache.hc.core5.net.URIBuilder; -import org.testcontainers.utility.ThrowingFunction; -import org.testng.Assert; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.net.InetAddress; -import java.net.Socket; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.Base64; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Pattern; -import java.util.zip.GZIPOutputStream; - -import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; -import static org.testng.Assert.fail; - -@Test(groups = {"integration"}) -public class HttpTransportTests extends BaseIntegrationTest { - - @Test(groups = {"integration"},dataProvider = "testConnectionTTLProvider") - @SuppressWarnings("java:S2925") - public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSockets) throws Exception { - if (isCloud()) { - return; // skip cloud tests because of wiremock proxy. TODO: fix it - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - int proxyPort = new Random().nextInt(1000) + 10000; - ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(proxyPort) - .networkTrafficListener(connectionCounter) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - URIBuilder targetURI = new URIBuilder(server.getBaseUri()) - .setPath(""); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().proxiedFrom(targetURI.build().toString())).build()); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(getPassword()) - .addProxy(ProxyType.HTTP, "localhost", proxyPort); - if (connectionTtl != null) { - clientBuilder.setConnectionTTL(connectionTtl, ChronoUnit.MILLIS); - } - if (keepAlive != null) { - clientBuilder.setKeepAliveTimeout(keepAlive, ChronoUnit.MILLIS); - } - - try (Client client = clientBuilder.build()) { - List resp = client.queryAll("select 1"); - Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); - - try { - Thread.sleep(1000L); - } catch (InterruptedException e) { - Assert.fail("Unexpected exception", e); - } - - resp = client.queryAll("select 1"); - Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - Assert.assertEquals(connectionCounter.opened.get(), openSockets); - proxy.stop(); - } - } - - @DataProvider(name = "testConnectionTTLProvider") - public static Object[][] testConnectionTTLProvider() { - return new Object[][] { - { 1000L, null, 2 }, - { 2000L, null, 1 }, - { null, 2000L, 1 }, - { null, 500L, 2 }, - { 1000L, 0L, 2 }, - { 1000L, 3000L, 2} - }; - } - - private static class ConnectionCounterListener implements WiremockNetworkTrafficListener { - - private AtomicInteger opened = new AtomicInteger(0); - private AtomicInteger closed = new AtomicInteger(0); - - @Override - public void opened(Socket socket) { - opened.incrementAndGet(); - } - - @Override - public void incoming(Socket socket, ByteBuffer bytes) { - // ignore - } - - @Override - public void outgoing(Socket socket, ByteBuffer bytes) { - // ignore - } - - @Override - public void closed(Socket socket) { - closed.incrementAndGet(); - } - } - - @Test(groups = {"integration"}) - public void testConnectionRequestTimeout() { - if (isCloud()) { - return; // mocked server - } - - int serverPort = new Random().nextInt(1000) + 10000; - ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(serverPort) - .networkTrafficListener(connectionCounter) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withFixedDelay(5000) - .withStatus(HttpStatus.SC_NOT_FOUND)).build()); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort) - .setUsername("default") - .setPassword(getPassword()) - .retryOnFailures(ClientFaultCause.None) - .setMaxConnections(1) - .setOption(ClickHouseClientOption.ASYNC.getKey(), "true") - .setSocketTimeout(10000, ChronoUnit.MILLIS) - .setConnectionRequestTimeout(5, ChronoUnit.MILLIS); - - try (Client client = clientBuilder.build()) { - CompletableFuture f1 = client.query("select 1"); - Thread.sleep(500L); - CompletableFuture f2 = client.query("select 1"); - f2.get(); - } catch (ExecutionException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCause().getClass(), ConnectionInitiationException.class); - Assert.assertEquals(e.getCause().getCause().getClass(), ConnectionRequestTimeoutException.class); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - proxy.stop(); - } - } - - @Test(groups = {"integration"}) - public void testConnectionReuseStrategy() { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(getPassword()) - .setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO) - .build()) { - - List records = client.queryAll("SELECT timezone()"); - Assert.assertTrue(records.size() > 0); - Assert.assertEquals(records.get(0).getString(1), "UTC"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test(groups = { "integration" }) - public void testSecureConnection() { - if (isCloud()) { - return; // will fail in other tests - } - - ClickHouseNode secureServer = getSecureServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder() - .addEndpoint("https://localhost:" + secureServer.getPort()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .compressClientRequest(true) - .build()) { - - List records = client.queryAll("SELECT timezone()"); - Assert.assertTrue(records.size() > 0); - Assert.assertEquals(records.get(0).getString(1), "UTC"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test(groups = { "integration" }, dataProvider = "NoResponseFailureProvider") - public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, ThrowingFunction function, - boolean shouldFail) { - if (isCloud()) { - return; // mocked server - } - - WireMockServer faultyServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - faultyServer.start(); - - // First request gets no response - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withRequestBody(WireMock.equalTo(body)) - .inScenario("Retry") - .whenScenarioStateIs(STARTED) - .willSetStateTo("Failed") - .willReturn(WireMock.aResponse().withFault(Fault.EMPTY_RESPONSE)).build()); - - // Second request gets a response (retry) - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withRequestBody(WireMock.equalTo(body)) - .inScenario("Retry") - .whenScenarioStateIs("Failed") - .willSetStateTo("Done") - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - Client mockServerClient = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setMaxRetries(maxRetries) - .build(); - - try { - function.apply(mockServerClient); - } catch (ConnectionInitiationException e) { - e.printStackTrace(); - if (!shouldFail) { - Assert.fail("Unexpected exception", e); - } - return; - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } finally { - faultyServer.stop(); - } - - if (shouldFail) { - Assert.fail("Expected exception"); - } - } - - @DataProvider(name = "NoResponseFailureProvider") - public static Object[][] noResponseFailureProvider() { - - String insertBody = "1\t2\t3\n"; - ThrowingFunction insertFunction = (client) -> { - InsertResponse insertResponse = client.insert("table01", - new ByteArrayInputStream("1\t2\t3\n".getBytes()), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS); - insertResponse.close(); - return null; - }; - - String selectBody = "select timezone()"; - ThrowingFunction queryFunction = (client) -> { - QueryResponse response = client.query("select timezone()").get(30, TimeUnit.SECONDS); - response.close(); - return null; - }; - - return new Object[][]{ - {insertBody, 1, insertFunction, false}, - {selectBody, 1, queryFunction, false}, - {insertBody, 0, insertFunction, true}, - {selectBody, 0, queryFunction, true} - }; - } - - @Test(groups = { "integration" }, dataProvider = "testServerErrorHandlingDataProvider") - public void testServerErrorHandling(ClickHouseFormat format, boolean serverCompression, boolean useHttpCompression) { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(serverCompression) - .useHttpCompression(useHttpCompression) - .build()) { - - QuerySettings querySettings = new QuerySettings().setFormat(format); - try (QueryResponse response = - client.query("SELECT invalid;statement", querySettings).get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 62); - Assert.assertTrue(e.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"), - "Unexpected error message: " + e.getMessage()); - } - - - try (QueryResponse response = client.query("CREATE TABLE table_from_csv ENGINE MergeTree ORDER BY () AS SELECT * FROM file('empty.csv') ", querySettings) - .get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 636); - Assert.assertTrue(e.getMessage().contains("You can specify the structure manually: (in file/uri /var/lib/clickhouse/user_files/empty.csv). (CANNOT_EXTRACT_TABLE_STRUCTURE)"), - "Unexpected error message: " + e.getMessage()); - } - - querySettings.serverSetting("unknown_setting", "1"); - try (QueryResponse response = client.query("CREATE TABLE table_from_csv AS SELECT * FROM file('empty.csv')", querySettings) - .get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 115); - Assert.assertTrue(e.getMessage().startsWith("Code: 115. DB::Exception: Setting unknown_setting is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings. (UNKNOWN_SETTING)"), - "Unexpected error message: " + e.getMessage()); - } - - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage(), e); - } - - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("non-existing-user") - .setPassword("nothing") - .compressServerResponse(serverCompression) - .useHttpCompression(useHttpCompression) - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 516); - Assert.assertTrue(e.getMessage().startsWith("Code: 516. DB::Exception: non-existing-user: Authentication failed: password is incorrect, or there is no user with such name. (AUTHENTICATION_FAILED)"), - e.getMessage()); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } - } - - @DataProvider(name = "testServerErrorHandlingDataProvider") - public static Object[][] testServerErrorHandlingDataProvider() { - EnumSet formats = EnumSet.of(ClickHouseFormat.CSV, ClickHouseFormat.TSV, - ClickHouseFormat.JSON, ClickHouseFormat.JSONCompact); - - int permutations = 3; - Object[][] result = new Object[formats.size() * permutations][]; - - int i = 0; - for (ClickHouseFormat format : formats) { - // format, server compression, http compression - result[i++] = new Object[]{format, false, false}; - result[i++] = new Object[]{format, true, false}; - result[i++] = new Object[]{format, true, true}; - } - - return result; - } - - @Test(groups = { "integration" }) - public void testErrorWithSuccessfulResponse() { - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withChunkedDribbleDelay(2, 200) - .withHeader("X-ClickHouse-Exception-Code", "241") - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")) - .build()); - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertTrue(e.getMessage().startsWith("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }, dataProvider = "testServerErrorsUncompressedDataProvider") - public void testServerErrorsUncompressed(int code, String message, String expectedMessage) { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withChunkedDribbleDelay(2, 200) - .withHeader("X-ClickHouse-Exception-Code", String.valueOf(code)) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(message)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), code); - Assert.assertTrue(e.getMessage().startsWith(expectedMessage)); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testServerErrorsUncompressedDataProvider") - public static Object[][] testServerErrorsUncompressedDataProvider() { - return new Object[][] { - { 241, "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB", - "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"}, - {900, "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full", - "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full"}, - {901, "Code: 901. DB::Exception: I write, erase, rewrite\n" + - "Erase again, and then\n" + - "A poppy blooms\n" + - " (by Katsushika Hokusai)", - "Code: 901. DB::Exception: I write, erase, rewrite " + - "Erase again, and then " + - "A poppy blooms" + - " (by Katsushika Hokusai)"} - }; - } - - @Test(groups = { "integration" }) - public void testAdditionalHeaders() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .httpHeader("X-ClickHouse-Test", "default_value") - .httpHeader("X-ClickHouse-Test-2", Arrays.asList("default_value1", "default_value2")) - .httpHeader("X-ClickHouse-Test-3", Arrays.asList("default_value1", "default_value2")) - .httpHeader("X-ClickHouse-Test-4", "default_value4") - .build()) { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("X-ClickHouse-Test", WireMock.equalTo("test")) - .withHeader("X-ClickHouse-Test-2", WireMock.equalTo("test1,test2")) - .withHeader("X-ClickHouse-Test-3", WireMock.equalTo("default_value1,default_value2")) - .withHeader("X-ClickHouse-Test-4", WireMock.equalTo("default_value4")) - - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - QuerySettings querySettings = new QuerySettings() - .httpHeader("X-ClickHouse-Test", "test") - .httpHeader("X-ClickHouse-Test-2", Arrays.asList("test1", "test2")); - - try (QueryResponse response = client.query("SELECT 1", querySettings).get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testServerSettings() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .serverSetting("max_threads", "10") - .serverSetting("async_insert", "1") - .serverSetting("roles", Arrays.asList("role1", "role2")) - .compressClientRequest(true) - .build()) { - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withQueryParam("max_threads", WireMock.equalTo("10")) - .withQueryParam("async_insert", WireMock.equalTo("3")) - .withQueryParam("roles", WireMock.equalTo("role3,role2")) - .withQueryParam("compress", WireMock.equalTo("0")) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - QuerySettings querySettings = new QuerySettings() - .serverSetting("max_threads", "10") - .serverSetting("async_insert", "3") - .serverSetting("roles", Arrays.asList("role3", "role2")) - .serverSetting("compress", "0"); - try (QueryResponse response = client.query("SELECT 1", querySettings).get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - mockServer.stop(); - } - } - } - - static { - if (Boolean.getBoolean("test.debug")) { - System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); - } - } - - @Test(groups = { "integration" }) - public void testSSLAuthentication() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .setUsername("dba") - .setPassword("dba") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH ssl_certificate CN 'some_user'").get()) { - } - } - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .useSSLAuthentication(true) - .setUsername("some_user") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .setClientCertificate("some_user.crt") - .setClientKey("some_user.key") - .compressServerResponse(false) - .build()) { - - try (QueryResponse resp = client.query("SELECT 1").get()) { - Assert.assertEquals(resp.getReadRows(), 1); - } - } - } - - @Test(groups = { "integration" }, dataProvider = "testPasswordAuthenticationProvider", dataProviderClass = HttpTransportTests.class) - public void testPasswordAuthentication(String identifyWith, String identifyBy, boolean failsWithHeaders) throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("dba") - .setPassword("dba") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { - } - } catch (Exception e) { - Assert.fail("Failed on setup", e); - } - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .build()) { - - Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); - } catch (Exception e) { - Assert.fail("Failed to authenticate", e); - } - - if (failsWithHeaders) { - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .useHTTPBasicAuth(false) - .build()) { - - Assert.expectThrows(ClientException.class, () -> - client.queryAll("SELECT user()").get(0).getString(1)); - - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } - } - } - - @DataProvider(name = "testPasswordAuthenticationProvider") - public static Object[][] testPasswordAuthenticationProvider() { - return new Object[][] { - { "plaintext_password", "password", false}, - { "plaintext_password", "", false }, - { "plaintext_password", "S3Cr=?t", true}, - { "plaintext_password", "123§", true }, - { "sha256_password", "password", false }, - { "sha256_password", "123§", true }, - { "sha256_password", "S3Cr=?t", true}, - { "sha256_password", "S3Cr?=t", false}, - }; - } - - @Test(groups = { "integration" }) - public void testAuthHeaderIsKeptFromUser() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - String identifyWith = "sha256_password"; - String identifyBy = "123§"; - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("dba") - .setPassword("dba") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { - } - } catch (Exception e) { - Assert.fail("Failed on setup", e); - } - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .useHTTPBasicAuth(false) // disable basic auth to produce CH headers - .httpHeader(HttpHeaders.AUTHORIZATION, "Basic " + Base64.getEncoder().encodeToString(("some_user:" +identifyBy).getBytes())) - .build()) { - - Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); - } catch (Exception e) { - Assert.fail("Failed to authenticate", e); - } - } - - @Test(groups = { "integration" }) - public void testSSLAuthentication_invalidConfig() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .useSSLAuthentication(true) - .setUsername("some_user") - .setPassword("s3cret") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .setClientCertificate("some_user.crt") - .setClientKey("some_user.key") - .compressServerResponse(false) - .build()) { - fail("Expected exception"); - } catch (IllegalArgumentException e) { - e.printStackTrace(); - Assert.assertTrue(e.getMessage().startsWith("Only one of password, access token or SSL authentication")); - } - } - - @Test(groups = { "integration" }) - public void testErrorWithSendProgressHeaders() throws Exception { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .build()) { - - try (CommandResponse resp = client.execute("DROP TABLE IF EXISTS test_omm_table").get()) { - } - try (CommandResponse resp = client.execute("CREATE TABLE test_omm_table ( val String) Engine = MergeTree ORDER BY () ").get()) { - } - - QuerySettings settings = new QuerySettings() - .serverSetting("send_progress_in_http_headers", "1") - .serverSetting("max_memory_usage", "54M"); - - try (QueryResponse resp = client.query("INSERT INTO test_omm_table SELECT randomString(16) FROM numbers(300000000)", settings).get()) { - - } catch (ServerException e) { - // 241 - MEMORY_LIMIT_EXCEEDED or 243 -NOT_ENOUGH_SPACE - Assert.assertTrue(e.getCode() == 241 || e.getCode() == 243); - } - } - } - - - @Test(groups = { "integration" }, dataProvider = "testUserAgentHasCompleteProductName_dataProvider", dataProviderClass = HttpTransportTests.class) - public void testUserAgentHasCompleteProductName(String clientName, Pattern userAgentPattern) throws Exception { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setClientName(clientName) - .build()) { - - String q1Id = UUID.randomUUID().toString(); - - client.execute("SELECT 1", (CommandSettings) new CommandSettings().setQueryId(q1Id)).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT http_user_agent, http_referer, " + - " forwarded_for FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + q1Id + "'"); - Assert.assertFalse(logRecords.isEmpty(), "No records found in query log"); - - for (GenericRecord record : logRecords) { - System.out.println(record.getString("http_user_agent")); - Assert.assertTrue(userAgentPattern.matcher(record.getString("http_user_agent")).matches(), - record.getString("http_user_agent") + " doesn't match \"" + - userAgentPattern.pattern() + "\""); - - } - } - } - - - @DataProvider(name = "testUserAgentHasCompleteProductName_dataProvider") - public static Object[][] testUserAgentHasCompleteProductName_dataProvider() { - return new Object[][] { - { "", Pattern.compile("clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$") }, - { "test-client/1.0", Pattern.compile("test-client/1.0 clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}, - { "test-client/", Pattern.compile("test-client/ clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}}; - } - - @Test(dataProvider = "testClientNameDataProvider") - public void testClientName(String clientName, boolean setWithUpdate, String userAgentHeader, boolean setForRequest) throws Exception { - - final String initialClientName = setWithUpdate ? "init clientName" : clientName; - final String initialUserAgentHeader = setForRequest ? "init userAgentHeader" : userAgentHeader; - final String clientReferer = "http://localhost/webpage"; - - Client.Builder builder = newClient(); - if (initialClientName != null) { - builder.setClientName(initialClientName); - } - if (initialUserAgentHeader != null) { - builder.httpHeader(HttpHeaders.USER_AGENT, initialUserAgentHeader); - } - try (Client client = builder.build()) { - String expectedClientNameStartsWith = initialClientName == null || initialUserAgentHeader != null ? initialUserAgentHeader : initialClientName; - - if (setWithUpdate) { - client.updateClientName(clientName); - expectedClientNameStartsWith = initialUserAgentHeader == null ? clientName : initialUserAgentHeader; - } - - String qId = UUID.randomUUID().toString(); - QuerySettings settings = new QuerySettings() - .httpHeader(HttpHeaders.REFERER, clientReferer) - .setQueryId(qId); - - if (setForRequest) { - settings.httpHeader(HttpHeaders.USER_AGENT, userAgentHeader); - expectedClientNameStartsWith = userAgentHeader; - } - - client.query("SELECT 1", settings).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + - " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); - Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); - final String logUserAgent = logRecords.get(0).getString("http_user_agent"); - Assert.assertTrue(logUserAgent.startsWith(expectedClientNameStartsWith), - "Expected to start with \"" + expectedClientNameStartsWith + "\" but values was \"" + logUserAgent + "\"" ); - Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); - Assert.assertEquals(logRecords.get(0).getString("http_referer"), clientReferer); - Assert.assertEquals(logRecords.get(0).getString("client_name"), ""); // http client can't set this field - } - } - - @DataProvider(name = "testClientNameDataProvider") - public static Object[][] testClientName() { - return new Object[][] { - {"test-product (app 1.0)", false, null, false}, // only client name set - {"test-product (app 1.0)", false, "final product (app 1.1)", false}, // http header set and overrides client name - {"test-product (app 1.0)", true, null, false}, // client name set thru Client#updateClientName - {"test-product (app 1.0)", true, "final product (app 1.1)", true}, // custom UserAgent header overrides client name - }; - } - - @Test(dataProvider = "testClientNameThruRawOptionsDataProvider") - public void testClientNameThruRawOptions(String property, String value, boolean setInClient) throws Exception { - Client.Builder builder = newClient(); - if (setInClient) { - builder.setOption(property, value); - } - try (Client client = builder.build()) { - - String qId = UUID.randomUUID().toString(); - QuerySettings settings = new QuerySettings() - .setQueryId(qId); - - if (!setInClient) { - settings.setOption(property, value); - } - - client.query("SELECT 1", settings).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + - " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); - Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); - final String logUserAgent = logRecords.get(0).getString("http_user_agent"); - Assert.assertTrue(logUserAgent.startsWith(value), - "Expected to start with \"" + value + "\" but values was \"" + logUserAgent + "\"" ); - Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); - } - } - - @DataProvider(name = "testClientNameThruRawOptionsDataProvider") - public Object[][] testClientNameThruRawOptionsDataProvider() { - return new Object[][] { - {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", true}, - {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", true}, - {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", false}, - {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", false}, - }; - } - - @Test(groups = { "integration" }) - public void testBearerTokenAuth() throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - String jwtToken1 = Arrays.stream( - new String[]{"header", "payload", "signature"}) - .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) - .reduce((s1, s2) -> s1 + "." + s2).get(); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(jwtToken1) - .compressServerResponse(false) - .build()) { - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } - } - - String jwtToken2 = Arrays.stream( - new String[]{"header2", "payload2", "signature2"}) - .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) - .reduce((s1, s2) -> s1 + "." + s2).get(); - - mockServer.resetAll(); - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_UNAUTHORIZED)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(jwtToken1) - .compressServerResponse(false) - .build()) { - - try { - client.execute("SELECT 1").get(); - fail("Exception expected"); - } catch (ServerException e) { - Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); - } - - mockServer.resetAll(); - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) - - .build()); - - client.updateBearerToken(jwtToken2); - - client.execute("SELECT 1").get(); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testBasicAuthWithNoPassword() throws Exception { - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - // Expected: "default:" with empty password, not "default:null" - String expectedAuth = "Basic " + Base64.getEncoder() - .encodeToString("default:".getBytes(StandardCharsets.UTF_8)); - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo(expectedAuth)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .compressServerResponse(false) - // no setPassword() call — password should default to empty, not "null" - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Basic auth with no password should send empty password, not 'null'", e); - } - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testJWTWithCloud() throws Exception { - if (!isCloud()) { - return; // only for cloud - } - String jwt = System.getenv("CLIENT_JWT"); - Assert.assertTrue(jwt != null && !jwt.trim().isEmpty(), "JWT is missing"); - Assert.assertFalse(jwt.contains("\n") || jwt.contains("-----"), "JWT should be single string ready for HTTP header"); - try (Client client = newClient().useBearerTokenAuth(jwt).build()) { - try { - List response = client.queryAll("SELECT user(), now()"); - System.out.println("response: " + response.get(0).getString(1) + " time: " + response.get(0).getString(2)); - } catch (Exception e) { - e.printStackTrace(); - throw e; - } - } - } - - @Test(groups = { "integration" }) - public void testWithDefaultTimeouts() { - if (isCloud()) { - return; // mocked server - } - - int proxyPort = new Random().nextInt(1000) + 10000; - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(proxyPort) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withFixedDelay(5000) - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", proxyPort, false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .build()) { - int startTime = (int) System.currentTimeMillis(); - try { - client.query("SELECT 1").get(); - } catch (Exception e) { - Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); - } - } finally { - proxy.stop(); - } - } - - - @Test(groups = { "integration" }) - public void testTimeoutsWithRetry() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer faultyServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - faultyServer.start(); - - // First request gets no response - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .inScenario("Timeout") - .withRequestBody(WireMock.containing("SELECT 1")) - .whenScenarioStateIs(STARTED) - .willSetStateTo("Failed") - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withFixedDelay(5000) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - // Second request gets a response (retry) - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .inScenario("Timeout") - .withRequestBody(WireMock.containing("SELECT 1")) - .whenScenarioStateIs("Failed") - .willSetStateTo("Done") - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withFixedDelay(1000) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setSocketTimeout(3000) - .retryOnFailures(ClientFaultCause.SocketTimeout) - .build()) { - int startTime = (int) System.currentTimeMillis(); - try { - client.query("SELECT 1").get(); - } catch (Exception e) { - Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); - } - } finally { - faultyServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testSNIWithCloud() throws Exception { - if (!isCloud()) { - // skip for local env - return; - } - - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - String ip = InetAddress.getByName(node.getHost()).getHostAddress(); - try (Client c = new Client.Builder() - .addEndpoint(Protocol.HTTP, ip, node.getPort(), true) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .sslSocketSNI(node.getHost()).build()) { - c.execute("SELECT 1"); - } - } - - @Test(groups = {"integration"}) - public void testEndpointUrlPathIsPreserved() throws Exception { - if (isCloud()) { - return; // mocked server - } - - int serverPort = new Random().nextInt(1000) + 10000; - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().port(serverPort) - .notifier(new Slf4jNotifier(true))); - mockServer.start(); - - try { - // Setup stubs for two virtual ClickHouse instances behind a reverse proxy - mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales/db")) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); - - mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing/db")) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); - - // Test sales virtual instance - try (Client salesClient = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort + "/sales/db") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = salesClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 100); - } - } - - // Test billing virtual instance - also verify query parameters in URL are ignored - try (Client billingClient = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort + "/billing/db?ignored_param=value") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = billingClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 200); - } - - // Verify that ignored_param is not in the request URL - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db")) - .withoutQueryParam("ignored_param")); - } - - // Verify requests were made to the correct paths - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales/db"))); - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db"))); - - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testMultiPartRequest() { - final Map params = new HashMap<>(); - params.put("database_name", "system"); - params.put("table_names", - DataTypeConverter.INSTANCE.arrayToString(Arrays.asList("COLLATIONS", "ENGINES"), "Array(String)")); - - // Use http compression - try (Client client = newClient().useHttpCompression(true).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - - // Use http compression - try (Client client = newClient().useHttpCompression(false).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - - // compress request - try (Client client = newClient() - .compressClientRequest(true) - .setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true") - .useHttpCompression(true).build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - } - - @Test(groups = {"integration"}) - public void testNotFoundError() { - if (isCloud()) { - return; // not needed - } - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint("http://" + node.getHost() + ":" + node.getPort() + "/some-path") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); - - try (Client client = clientBuilder.build()) { - client.queryAll("select 1"); - fail("Exception expected"); - } catch (ClientException e) { - Assert.assertTrue(e.getCause().getMessage().startsWith("There is no handle /some-path?")); - } - - } - - @Test(groups = {"integration"}) - public void testSmallNetworkBufferDoesNotBreakColumnDecoding() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 2_000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - - Assert.assertTrue(validBody.length > 3, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - 5); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, - "Expected failure when reading truncated binary stream"); - assertBinaryReadFailureContainsColumnName(thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testChunkedResponsePrematureEndIsReported() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT 10"; - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader(HttpHeaders.TRANSFER_ENCODING, "chunked") - .withFault(Fault.MALFORMED_RESPONSE_CHUNK)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, null, 30, - "Expected failure when reading malformed chunked response"); - ConnectionClosedException connectionClosedException = findCause(thrown, ConnectionClosedException.class); - boolean hasChunkedPrematureCloseSignature = containsMessageInCauseChain(thrown, - "closing chunk expected", - "premature end of chunk coded message body", - "failed to read header"); - Assert.assertTrue(connectionClosedException != null || hasChunkedPrematureCloseSignature, - "Expected chunked/premature-close failure signature, but was: " + thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testTailCorruptedStreamFailsDecoding() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 100_000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - - final int removedBytes = 3; - Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, - "Expected failure when reading binary stream truncated at tail"); - assertBinaryReadFailureContainsColumnName(thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testTailStreamFailureReportsPositiveTimeSinceLastNextCall() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 2000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - final int removedBytes = 3; - Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newMockServerClient(mockServer.port(), networkBufferSize) - .build(); - QueryResponse response = client.query(query, querySettings).get(60, TimeUnit.SECONDS); - ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { - final int[] rowsRead = new int[] {0}; - Throwable thrown = Assert.expectThrows(Throwable.class, () -> { - while (true) { - if (rowsRead[0] >= 5) { - try { - Thread.sleep(25); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - if (reader.next() == null) { - return; - } - rowsRead[0]++; - } - }); - - Assert.assertTrue(rowsRead[0] >= 5, - "Expected to read at least a few rows before failure, but read " + rowsRead[0]); - ClientException clientException = findCause(thrown, ClientException.class); - Assert.assertNotNull(clientException, - "Expected ClientException in cause chain, but was: " + thrown); - Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), - "Expected column information in failure message chain, but was: " + thrown); - - String elapsedTimeMessage = findFirstMessageInCauseChain(thrown, "time since last next call"); - Assert.assertNotNull(elapsedTimeMessage, - "Expected elapsed-time fragment in failure message chain, but was: " + thrown); - - java.util.regex.Matcher matcher = Pattern.compile("time since last next call (\\d+)\\)") - .matcher(elapsedTimeMessage); - Assert.assertTrue(matcher.find(), - "Expected elapsed-time fragment in message: " + elapsedTimeMessage); - long elapsedSinceLastNext = Long.parseLong(matcher.group(1)); - Assert.assertTrue(elapsedSinceLastNext > 0, - "Expected positive elapsed time since last next call, but was " + elapsedSinceLastNext); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") - public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(httpStatus) - .withBody(responseBody)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), - "Expected to contain '" + expectedBodyPart + "', but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusErrorBodyDataProvider") - public static Object[][] testHttpStatusErrorBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: invalid credentials for user default", "invalid credentials"}, - {HttpStatus.SC_FORBIDDEN, "Forbidden: user default has no access to this operation", "no access"}, - {HttpStatus.SC_NOT_FOUND, "Not found: requested endpoint does not exist", "requested endpoint"} - }; - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusWithoutBodyDataProvider") - public void testHttpStatusErrorsWithoutBody(int httpStatus) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withStatus(httpStatus)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains("unknown server error"), - "Expected unknown error message for empty body, but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusWithoutBodyDataProvider") - public static Object[][] testHttpStatusWithoutBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED}, - {HttpStatus.SC_FORBIDDEN}, - {HttpStatus.SC_NOT_FOUND} - }; - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusCompressedBodyDataProvider") - public void testHttpStatusErrorsWithHttpCompression(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(httpStatus) - .withHeader(HttpHeaders.CONTENT_ENCODING, "gzip") - .withBody(gzip(responseBody))) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .useHttpCompression(true) - .compressServerResponse(true) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), - "Expected compressed body part '" + expectedBodyPart + "', but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusCompressedBodyDataProvider") - public static Object[][] testHttpStatusCompressedBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: token is expired", "token is expired"}, - {HttpStatus.SC_FORBIDDEN, "Forbidden: policy denies this query", "policy denies"}, - {HttpStatus.SC_NOT_FOUND, "Not found: route does not exist", "route does not exist"} - }; - } - - private byte[] fetchBinaryPayload(String query, int networkBufferSize, int timeoutSec) throws Exception { - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newClient() - .useHttpCompression(false) - .compressServerResponse(false) - .setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)) - .build(); - QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS)) { - return readAllBytes(response.getInputStream()); - } - } - - private Throwable assertBinaryDecodeFails(WireMockServer mockServer, String query, Integer networkBufferSize, - int timeoutSec, String assertionMessage) throws Exception { - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newMockServerClient(mockServer.port(), networkBufferSize).build()) { - Throwable thrown = Assert.expectThrows(Throwable.class, () -> { - try (QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS); - ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { - readAllRows(reader); - } - }); - Assert.assertNotNull(thrown, assertionMessage); - return thrown; - } - } - - private Client.Builder newMockServerClient(int port, Integer networkBufferSize) { - Client.Builder builder = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", port, false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1") - .useHttpCompression(false) - .compressServerResponse(false); - if (networkBufferSize != null) { - builder.setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)); - } - return builder; - } - - private static void readAllRows(ClickHouseBinaryFormatReader reader) { - while (reader.next() != null) { - reader.getInteger(1); - reader.getLong(2); - reader.getString(3); - } - } - - private static byte[] gzip(String value) throws Exception { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { - gzipOutputStream.write(value.getBytes(StandardCharsets.UTF_8)); - } - return out.toByteArray(); - } - - private static byte[] readAllBytes(java.io.InputStream inputStream) throws Exception { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - byte[] buffer = new byte[8192]; - int read; - while ((read = inputStream.read(buffer)) != -1) { - out.write(buffer, 0, read); - } - return out.toByteArray(); - } - - private static ServerException findServerException(Throwable throwable) { - Throwable current = throwable; - while (current != null) { - if (current instanceof ServerException) { - return (ServerException) current; - } - current = current.getCause(); - } - return null; - } - - private static T findCause(Throwable throwable, Class clazz) { - Throwable current = throwable; - while (current != null) { - if (clazz.isInstance(current)) { - return clazz.cast(current); - } - current = current.getCause(); - } - return null; - } - - private static boolean containsMessageInCauseChain(Throwable throwable, String... parts) { - Throwable current = throwable; - while (current != null) { - String message = current.getMessage(); - if (message != null) { - String lower = message.toLowerCase(); - for (String part : parts) { - if (lower.contains(part.toLowerCase())) { - return true; - } - } - } - current = current.getCause(); - } - return false; - } - - private static String findFirstMessageInCauseChain(Throwable throwable, String part) { - Throwable current = throwable; - while (current != null) { - String message = current.getMessage(); - if (message != null && message.contains(part)) { - return message; - } - current = current.getCause(); - } - return null; - } - - private static void assertBinaryReadFailureContainsColumnName(Throwable thrown) { - thrown.printStackTrace(); - ClientException clientException = findCause(thrown, ClientException.class); - Assert.assertNotNull(clientException, - "Expected ClientException in cause chain, but was: " + thrown); - Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), - "Expected column information in failure message chain, but was: " + thrown); - } - - protected Client.Builder newClient() { - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - boolean isSecure = isCloud(); - return new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); - } -} +package com.clickhouse.client; + +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.ClientFaultCause; +import com.clickhouse.client.api.ConnectionInitiationException; +import com.clickhouse.client.api.ConnectionReuseStrategy; +import com.clickhouse.client.api.ServerException; +import com.clickhouse.client.api.command.CommandResponse; +import com.clickhouse.client.api.command.CommandSettings; +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.enums.ProxyType; +import com.clickhouse.client.api.insert.InsertResponse; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.client.config.ClickHouseClientOption; +import com.clickhouse.data.ClickHouseFormat; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.client.WireMock; +import com.github.tomakehurst.wiremock.common.ConsoleNotifier; +import com.github.tomakehurst.wiremock.common.Slf4jNotifier; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.http.Fault; +import com.github.tomakehurst.wiremock.http.trafficlistener.WiremockNetworkTrafficListener; +import org.apache.hc.core5.http.ConnectionClosedException; +import org.apache.hc.core5.http.ConnectionRequestTimeoutException; +import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.net.URIBuilder; +import org.testcontainers.utility.ThrowingFunction; +import org.testng.Assert; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.net.InetAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Base64; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.zip.GZIPOutputStream; + +import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; +import static org.testng.Assert.fail; + +@Test(groups = {"integration"}) +public class HttpTransportTests extends BaseIntegrationTest { + + @Test(groups = {"integration"},dataProvider = "testConnectionTTLProvider") + @SuppressWarnings("java:S2925") + public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSockets) throws Exception { + if (isCloud()) { + return; // skip cloud tests because of wiremock proxy. TODO: fix it + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + int proxyPort = new Random().nextInt(1000) + 10000; + ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(proxyPort) + .networkTrafficListener(connectionCounter) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + URIBuilder targetURI = new URIBuilder(server.getBaseUri()) + .setPath(""); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().proxiedFrom(targetURI.build().toString())).build()); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(getPassword()) + .addProxy(ProxyType.HTTP, "localhost", proxyPort); + if (connectionTtl != null) { + clientBuilder.setConnectionTTL(connectionTtl, ChronoUnit.MILLIS); + } + if (keepAlive != null) { + clientBuilder.setKeepAliveTimeout(keepAlive, ChronoUnit.MILLIS); + } + + try (Client client = clientBuilder.build()) { + List resp = client.queryAll("select 1"); + Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); + + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + Assert.fail("Unexpected exception", e); + } + + resp = client.queryAll("select 1"); + Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + Assert.assertEquals(connectionCounter.opened.get(), openSockets); + proxy.stop(); + } + } + + @DataProvider(name = "testConnectionTTLProvider") + public static Object[][] testConnectionTTLProvider() { + return new Object[][] { + { 1000L, null, 2 }, + { 2000L, null, 1 }, + { null, 2000L, 1 }, + { null, 500L, 2 }, + { 1000L, 0L, 2 }, + { 1000L, 3000L, 2} + }; + } + + private static class ConnectionCounterListener implements WiremockNetworkTrafficListener { + + private AtomicInteger opened = new AtomicInteger(0); + private AtomicInteger closed = new AtomicInteger(0); + + @Override + public void opened(Socket socket) { + opened.incrementAndGet(); + } + + @Override + public void incoming(Socket socket, ByteBuffer bytes) { + // ignore + } + + @Override + public void outgoing(Socket socket, ByteBuffer bytes) { + // ignore + } + + @Override + public void closed(Socket socket) { + closed.incrementAndGet(); + } + } + + @Test(groups = {"integration"}) + public void testConnectionRequestTimeout() { + if (isCloud()) { + return; // mocked server + } + + int serverPort = new Random().nextInt(1000) + 10000; + ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(serverPort) + .networkTrafficListener(connectionCounter) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withFixedDelay(5000) + .withStatus(HttpStatus.SC_NOT_FOUND)).build()); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort) + .setUsername("default") + .setPassword(getPassword()) + .retryOnFailures(ClientFaultCause.None) + .setMaxConnections(1) + .setOption(ClickHouseClientOption.ASYNC.getKey(), "true") + .setSocketTimeout(10000, ChronoUnit.MILLIS) + .setConnectionRequestTimeout(5, ChronoUnit.MILLIS); + + try (Client client = clientBuilder.build()) { + CompletableFuture f1 = client.query("select 1"); + Thread.sleep(500L); + CompletableFuture f2 = client.query("select 1"); + f2.get(); + } catch (ExecutionException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCause().getClass(), ConnectionInitiationException.class); + Assert.assertEquals(e.getCause().getCause().getClass(), ConnectionRequestTimeoutException.class); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + proxy.stop(); + } + } + + @Test(groups = {"integration"}) + public void testConnectionReuseStrategy() { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(getPassword()) + .setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO) + .build()) { + + List records = client.queryAll("SELECT timezone()"); + Assert.assertTrue(records.size() > 0); + Assert.assertEquals(records.get(0).getString(1), "UTC"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } + + @Test(groups = { "integration" }) + public void testSecureConnection() { + if (isCloud()) { + return; // will fail in other tests + } + + ClickHouseNode secureServer = getSecureServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder() + .addEndpoint("https://localhost:" + secureServer.getPort()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .compressClientRequest(true) + .build()) { + + List records = client.queryAll("SELECT timezone()"); + Assert.assertTrue(records.size() > 0); + Assert.assertEquals(records.get(0).getString(1), "UTC"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } + + @Test(groups = { "integration" }, dataProvider = "NoResponseFailureProvider") + public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, ThrowingFunction function, + boolean shouldFail) { + if (isCloud()) { + return; // mocked server + } + + WireMockServer faultyServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + faultyServer.start(); + + // First request gets no response + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withRequestBody(WireMock.equalTo(body)) + .inScenario("Retry") + .whenScenarioStateIs(STARTED) + .willSetStateTo("Failed") + .willReturn(WireMock.aResponse().withFault(Fault.EMPTY_RESPONSE)).build()); + + // Second request gets a response (retry) + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withRequestBody(WireMock.equalTo(body)) + .inScenario("Retry") + .whenScenarioStateIs("Failed") + .willSetStateTo("Done") + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + Client mockServerClient = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setMaxRetries(maxRetries) + .build(); + + try { + function.apply(mockServerClient); + } catch (ConnectionInitiationException e) { + e.printStackTrace(); + if (!shouldFail) { + Assert.fail("Unexpected exception", e); + } + return; + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } finally { + faultyServer.stop(); + } + + if (shouldFail) { + Assert.fail("Expected exception"); + } + } + + @DataProvider(name = "NoResponseFailureProvider") + public static Object[][] noResponseFailureProvider() { + + String insertBody = "1\t2\t3\n"; + ThrowingFunction insertFunction = (client) -> { + InsertResponse insertResponse = client.insert("table01", + new ByteArrayInputStream("1\t2\t3\n".getBytes()), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS); + insertResponse.close(); + return null; + }; + + String selectBody = "select timezone()"; + ThrowingFunction queryFunction = (client) -> { + QueryResponse response = client.query("select timezone()").get(30, TimeUnit.SECONDS); + response.close(); + return null; + }; + + return new Object[][]{ + {insertBody, 1, insertFunction, false}, + {selectBody, 1, queryFunction, false}, + {insertBody, 0, insertFunction, true}, + {selectBody, 0, queryFunction, true} + }; + } + + @Test(groups = { "integration" }, dataProvider = "testServerErrorHandlingDataProvider") + public void testServerErrorHandling(ClickHouseFormat format, boolean serverCompression, boolean useHttpCompression) { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(serverCompression) + .useHttpCompression(useHttpCompression) + .build()) { + + QuerySettings querySettings = new QuerySettings().setFormat(format); + try (QueryResponse response = + client.query("SELECT invalid;statement", querySettings).get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 62); + Assert.assertTrue(e.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"), + "Unexpected error message: " + e.getMessage()); + } + + + try (QueryResponse response = client.query("CREATE TABLE table_from_csv ENGINE MergeTree ORDER BY () AS SELECT * FROM file('empty.csv') ", querySettings) + .get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 636); + Assert.assertTrue(e.getMessage().contains("You can specify the structure manually: (in file/uri /var/lib/clickhouse/user_files/empty.csv). (CANNOT_EXTRACT_TABLE_STRUCTURE)"), + "Unexpected error message: " + e.getMessage()); + } + + querySettings.serverSetting("unknown_setting", "1"); + try (QueryResponse response = client.query("CREATE TABLE table_from_csv AS SELECT * FROM file('empty.csv')", querySettings) + .get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 115); + Assert.assertTrue(e.getMessage().startsWith("Code: 115. DB::Exception: Setting unknown_setting is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings. (UNKNOWN_SETTING)"), + "Unexpected error message: " + e.getMessage()); + } + + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage(), e); + } + + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("non-existing-user") + .setPassword("nothing") + .compressServerResponse(serverCompression) + .useHttpCompression(useHttpCompression) + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 516); + Assert.assertTrue(e.getMessage().startsWith("Code: 516. DB::Exception: non-existing-user: Authentication failed: password is incorrect, or there is no user with such name. (AUTHENTICATION_FAILED)"), + e.getMessage()); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } + } + + @DataProvider(name = "testServerErrorHandlingDataProvider") + public static Object[][] testServerErrorHandlingDataProvider() { + EnumSet formats = EnumSet.of(ClickHouseFormat.CSV, ClickHouseFormat.TSV, + ClickHouseFormat.JSON, ClickHouseFormat.JSONCompact); + + int permutations = 3; + Object[][] result = new Object[formats.size() * permutations][]; + + int i = 0; + for (ClickHouseFormat format : formats) { + // format, server compression, http compression + result[i++] = new Object[]{format, false, false}; + result[i++] = new Object[]{format, true, false}; + result[i++] = new Object[]{format, true, true}; + } + + return result; + } + + @Test(groups = { "integration" }) + public void testErrorWithSuccessfulResponse() { + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withChunkedDribbleDelay(2, 200) + .withHeader("X-ClickHouse-Exception-Code", "241") + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")) + .build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertTrue(e.getMessage().startsWith("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }, dataProvider = "testServerErrorsUncompressedDataProvider") + public void testServerErrorsUncompressed(int code, String message, String expectedMessage) { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withChunkedDribbleDelay(2, 200) + .withHeader("X-ClickHouse-Exception-Code", String.valueOf(code)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(message)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), code); + Assert.assertTrue(e.getMessage().startsWith(expectedMessage), "but started with " + e.getMessage()); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testServerErrorsUncompressedDataProvider") + public static Object[][] testServerErrorsUncompressedDataProvider() { + return new Object[][] { + { 241, "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB", + "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"}, + {900, "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full", + "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full"}, + {901, "Code: 901. DB::Exception: I write, erase, rewrite\n" + + "Erase again, and then\n" + + "A poppy blooms\n" + + " (by Katsushika Hokusai)", + "Code: 901. DB::Exception: I write, erase, rewrite " + + "Erase again, and then " + + "A poppy blooms" + + " (by Katsushika Hokusai)"} + }; + } + + @Test(groups = { "integration" }) + public void testAdditionalHeaders() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .httpHeader("X-ClickHouse-Test", "default_value") + .httpHeader("X-ClickHouse-Test-2", Arrays.asList("default_value1", "default_value2")) + .httpHeader("X-ClickHouse-Test-3", Arrays.asList("default_value1", "default_value2")) + .httpHeader("X-ClickHouse-Test-4", "default_value4") + .build()) { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("X-ClickHouse-Test", WireMock.equalTo("test")) + .withHeader("X-ClickHouse-Test-2", WireMock.equalTo("test1,test2")) + .withHeader("X-ClickHouse-Test-3", WireMock.equalTo("default_value1,default_value2")) + .withHeader("X-ClickHouse-Test-4", WireMock.equalTo("default_value4")) + + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + QuerySettings querySettings = new QuerySettings() + .httpHeader("X-ClickHouse-Test", "test") + .httpHeader("X-ClickHouse-Test-2", Arrays.asList("test1", "test2")); + + try (QueryResponse response = client.query("SELECT 1", querySettings).get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testServerSettings() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .serverSetting("max_threads", "10") + .serverSetting("async_insert", "1") + .serverSetting("roles", Arrays.asList("role1", "role2")) + .compressClientRequest(true) + .build()) { + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withQueryParam("max_threads", WireMock.equalTo("10")) + .withQueryParam("async_insert", WireMock.equalTo("3")) + .withQueryParam("roles", WireMock.equalTo("role3,role2")) + .withQueryParam("compress", WireMock.equalTo("0")) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + QuerySettings querySettings = new QuerySettings() + .serverSetting("max_threads", "10") + .serverSetting("async_insert", "3") + .serverSetting("roles", Arrays.asList("role3", "role2")) + .serverSetting("compress", "0"); + try (QueryResponse response = client.query("SELECT 1", querySettings).get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + mockServer.stop(); + } + } + } + + static { + if (Boolean.getBoolean("test.debug")) { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); + } + } + + @Test(groups = { "integration" }) + public void testSSLAuthentication() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .setUsername("dba") + .setPassword("dba") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH ssl_certificate CN 'some_user'").get()) { + } + } + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .useSSLAuthentication(true) + .setUsername("some_user") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .setClientCertificate("some_user.crt") + .setClientKey("some_user.key") + .compressServerResponse(false) + .build()) { + + try (QueryResponse resp = client.query("SELECT 1").get()) { + Assert.assertEquals(resp.getReadRows(), 1); + } + } + } + + @Test(groups = { "integration" }, dataProvider = "testPasswordAuthenticationProvider", dataProviderClass = HttpTransportTests.class) + public void testPasswordAuthentication(String identifyWith, String identifyBy, boolean failsWithHeaders) throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("dba") + .setPassword("dba") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { + } + } catch (Exception e) { + Assert.fail("Failed on setup", e); + } + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .build()) { + + Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); + } catch (Exception e) { + Assert.fail("Failed to authenticate", e); + } + + if (failsWithHeaders) { + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .useHTTPBasicAuth(false) + .build()) { + + Assert.expectThrows(ClientException.class, () -> + client.queryAll("SELECT user()").get(0).getString(1)); + + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + } + + @DataProvider(name = "testPasswordAuthenticationProvider") + public static Object[][] testPasswordAuthenticationProvider() { + return new Object[][] { + { "plaintext_password", "password", false}, + { "plaintext_password", "", false }, + { "plaintext_password", "S3Cr=?t", true}, + { "plaintext_password", "123§", true }, + { "sha256_password", "password", false }, + { "sha256_password", "123§", true }, + { "sha256_password", "S3Cr=?t", true}, + { "sha256_password", "S3Cr?=t", false}, + }; + } + + @Test(groups = { "integration" }) + public void testAuthHeaderIsKeptFromUser() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + String identifyWith = "sha256_password"; + String identifyBy = "123§"; + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("dba") + .setPassword("dba") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { + } + } catch (Exception e) { + Assert.fail("Failed on setup", e); + } + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .useHTTPBasicAuth(false) // disable basic auth to produce CH headers + .httpHeader(HttpHeaders.AUTHORIZATION, "Basic " + Base64.getEncoder().encodeToString(("some_user:" +identifyBy).getBytes())) + .build()) { + + Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); + } catch (Exception e) { + Assert.fail("Failed to authenticate", e); + } + } + + @Test(groups = { "integration" }) + public void testSSLAuthentication_invalidConfig() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .useSSLAuthentication(true) + .setUsername("some_user") + .setPassword("s3cret") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .setClientCertificate("some_user.crt") + .setClientKey("some_user.key") + .compressServerResponse(false) + .build()) { + fail("Expected exception"); + } catch (IllegalArgumentException e) { + e.printStackTrace(); + Assert.assertTrue(e.getMessage().startsWith("Only one of password, access token or SSL authentication")); + } + } + + @Test(groups = { "integration" }) + public void testErrorWithSendProgressHeaders() throws Exception { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + + try (CommandResponse resp = client.execute("DROP TABLE IF EXISTS test_omm_table").get()) { + } + try (CommandResponse resp = client.execute("CREATE TABLE test_omm_table ( val String) Engine = MergeTree ORDER BY () ").get()) { + } + + QuerySettings settings = new QuerySettings() + .serverSetting("send_progress_in_http_headers", "1") + .serverSetting("max_memory_usage", "54M"); + + try (QueryResponse resp = client.query("INSERT INTO test_omm_table SELECT randomString(16) FROM numbers(300000000)", settings).get()) { + + } catch (ServerException e) { + // 241 - MEMORY_LIMIT_EXCEEDED or 243 -NOT_ENOUGH_SPACE + Assert.assertTrue(e.getCode() == 241 || e.getCode() == 243); + } + } + } + + + @Test(groups = { "integration" }, dataProvider = "testUserAgentHasCompleteProductName_dataProvider", dataProviderClass = HttpTransportTests.class) + public void testUserAgentHasCompleteProductName(String clientName, Pattern userAgentPattern) throws Exception { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setClientName(clientName) + .build()) { + + String q1Id = UUID.randomUUID().toString(); + + client.execute("SELECT 1", (CommandSettings) new CommandSettings().setQueryId(q1Id)).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT http_user_agent, http_referer, " + + " forwarded_for FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + q1Id + "'"); + Assert.assertFalse(logRecords.isEmpty(), "No records found in query log"); + + for (GenericRecord record : logRecords) { + System.out.println(record.getString("http_user_agent")); + Assert.assertTrue(userAgentPattern.matcher(record.getString("http_user_agent")).matches(), + record.getString("http_user_agent") + " doesn't match \"" + + userAgentPattern.pattern() + "\""); + + } + } + } + + + @DataProvider(name = "testUserAgentHasCompleteProductName_dataProvider") + public static Object[][] testUserAgentHasCompleteProductName_dataProvider() { + return new Object[][] { + { "", Pattern.compile("clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$") }, + { "test-client/1.0", Pattern.compile("test-client/1.0 clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}, + { "test-client/", Pattern.compile("test-client/ clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}}; + } + + @Test(dataProvider = "testClientNameDataProvider") + public void testClientName(String clientName, boolean setWithUpdate, String userAgentHeader, boolean setForRequest) throws Exception { + + final String initialClientName = setWithUpdate ? "init clientName" : clientName; + final String initialUserAgentHeader = setForRequest ? "init userAgentHeader" : userAgentHeader; + final String clientReferer = "http://localhost/webpage"; + + Client.Builder builder = newClient(); + if (initialClientName != null) { + builder.setClientName(initialClientName); + } + if (initialUserAgentHeader != null) { + builder.httpHeader(HttpHeaders.USER_AGENT, initialUserAgentHeader); + } + try (Client client = builder.build()) { + String expectedClientNameStartsWith = initialClientName == null || initialUserAgentHeader != null ? initialUserAgentHeader : initialClientName; + + if (setWithUpdate) { + client.updateClientName(clientName); + expectedClientNameStartsWith = initialUserAgentHeader == null ? clientName : initialUserAgentHeader; + } + + String qId = UUID.randomUUID().toString(); + QuerySettings settings = new QuerySettings() + .httpHeader(HttpHeaders.REFERER, clientReferer) + .setQueryId(qId); + + if (setForRequest) { + settings.httpHeader(HttpHeaders.USER_AGENT, userAgentHeader); + expectedClientNameStartsWith = userAgentHeader; + } + + client.query("SELECT 1", settings).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + + " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); + Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); + final String logUserAgent = logRecords.get(0).getString("http_user_agent"); + Assert.assertTrue(logUserAgent.startsWith(expectedClientNameStartsWith), + "Expected to start with \"" + expectedClientNameStartsWith + "\" but values was \"" + logUserAgent + "\"" ); + Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); + Assert.assertEquals(logRecords.get(0).getString("http_referer"), clientReferer); + Assert.assertEquals(logRecords.get(0).getString("client_name"), ""); // http client can't set this field + } + } + + @DataProvider(name = "testClientNameDataProvider") + public static Object[][] testClientName() { + return new Object[][] { + {"test-product (app 1.0)", false, null, false}, // only client name set + {"test-product (app 1.0)", false, "final product (app 1.1)", false}, // http header set and overrides client name + {"test-product (app 1.0)", true, null, false}, // client name set thru Client#updateClientName + {"test-product (app 1.0)", true, "final product (app 1.1)", true}, // custom UserAgent header overrides client name + }; + } + + @Test(dataProvider = "testClientNameThruRawOptionsDataProvider") + public void testClientNameThruRawOptions(String property, String value, boolean setInClient) throws Exception { + Client.Builder builder = newClient(); + if (setInClient) { + builder.setOption(property, value); + } + try (Client client = builder.build()) { + + String qId = UUID.randomUUID().toString(); + QuerySettings settings = new QuerySettings() + .setQueryId(qId); + + if (!setInClient) { + settings.setOption(property, value); + } + + client.query("SELECT 1", settings).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + + " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); + Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); + final String logUserAgent = logRecords.get(0).getString("http_user_agent"); + Assert.assertTrue(logUserAgent.startsWith(value), + "Expected to start with \"" + value + "\" but values was \"" + logUserAgent + "\"" ); + Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); + } + } + + @DataProvider(name = "testClientNameThruRawOptionsDataProvider") + public Object[][] testClientNameThruRawOptionsDataProvider() { + return new Object[][] { + {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", true}, + {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", true}, + {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", false}, + {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", false}, + }; + } + + @Test(groups = { "integration" }) + public void testBearerTokenAuth() throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + String jwtToken1 = Arrays.stream( + new String[]{"header", "payload", "signature"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) + .build()) { + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + String jwtToken2 = Arrays.stream( + new String[]{"header2", "payload2", "signature2"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); + + mockServer.resetAll(); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_UNAUTHORIZED)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) + .build()) { + + try { + client.execute("SELECT 1").get(); + fail("Exception expected"); + } catch (ServerException e) { + Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); + } + + mockServer.resetAll(); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) + + .build()); + + client.updateBearerToken(jwtToken2); + + client.execute("SELECT 1").get(); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testBasicAuthWithNoPassword() throws Exception { + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + // Expected: "default:" with empty password, not "default:null" + String expectedAuth = "Basic " + Base64.getEncoder() + .encodeToString("default:".getBytes(StandardCharsets.UTF_8)); + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo(expectedAuth)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .compressServerResponse(false) + // no setPassword() call — password should default to empty, not "null" + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Basic auth with no password should send empty password, not 'null'", e); + } + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testJWTWithCloud() throws Exception { + if (!isCloud()) { + return; // only for cloud + } + String jwt = System.getenv("CLIENT_JWT"); + Assert.assertTrue(jwt != null && !jwt.trim().isEmpty(), "JWT is missing"); + Assert.assertFalse(jwt.contains("\n") || jwt.contains("-----"), "JWT should be single string ready for HTTP header"); + try (Client client = newClient().useBearerTokenAuth(jwt).build()) { + try { + List response = client.queryAll("SELECT user(), now()"); + System.out.println("response: " + response.get(0).getString(1) + " time: " + response.get(0).getString(2)); + } catch (Exception e) { + e.printStackTrace(); + throw e; + } + } + } + + @Test(groups = { "integration" }) + public void testWithDefaultTimeouts() { + if (isCloud()) { + return; // mocked server + } + + int proxyPort = new Random().nextInt(1000) + 10000; + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(proxyPort) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withFixedDelay(5000) + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", proxyPort, false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + int startTime = (int) System.currentTimeMillis(); + try { + client.query("SELECT 1").get(); + } catch (Exception e) { + Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); + } + } finally { + proxy.stop(); + } + } + + + @Test(groups = { "integration" }) + public void testTimeoutsWithRetry() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer faultyServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + faultyServer.start(); + + // First request gets no response + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .inScenario("Timeout") + .withRequestBody(WireMock.containing("SELECT 1")) + .whenScenarioStateIs(STARTED) + .willSetStateTo("Failed") + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withFixedDelay(5000) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + // Second request gets a response (retry) + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .inScenario("Timeout") + .withRequestBody(WireMock.containing("SELECT 1")) + .whenScenarioStateIs("Failed") + .willSetStateTo("Done") + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withFixedDelay(1000) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setSocketTimeout(3000) + .retryOnFailures(ClientFaultCause.SocketTimeout) + .build()) { + int startTime = (int) System.currentTimeMillis(); + try { + client.query("SELECT 1").get(); + } catch (Exception e) { + Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); + } + } finally { + faultyServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testSNIWithCloud() throws Exception { + if (!isCloud()) { + // skip for local env + return; + } + + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + String ip = InetAddress.getByName(node.getHost()).getHostAddress(); + try (Client c = new Client.Builder() + .addEndpoint(Protocol.HTTP, ip, node.getPort(), true) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .sslSocketSNI(node.getHost()).build()) { + c.execute("SELECT 1"); + } + } + + @Test(groups = {"integration"}) + public void testEndpointUrlPathIsPreserved() throws Exception { + if (isCloud()) { + return; // mocked server + } + + int serverPort = new Random().nextInt(1000) + 10000; + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(serverPort) + .notifier(new Slf4jNotifier(true))); + mockServer.start(); + + try { + // Setup stubs for two virtual ClickHouse instances behind a reverse proxy + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); + + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); + + // Test sales virtual instance + try (Client salesClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/sales/db") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = salesClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 100); + } + } + + // Test billing virtual instance - also verify query parameters in URL are ignored + try (Client billingClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/billing/db?ignored_param=value") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = billingClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 200); + } + + // Verify that ignored_param is not in the request URL + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db")) + .withoutQueryParam("ignored_param")); + } + + // Verify requests were made to the correct paths + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales/db"))); + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db"))); + + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testMultiPartRequest() { + final Map params = new HashMap<>(); + params.put("database_name", "system"); + params.put("table_names", + DataTypeConverter.INSTANCE.arrayToString(Arrays.asList("COLLATIONS", "ENGINES"), "Array(String)")); + + // Use http compression + try (Client client = newClient().useHttpCompression(true).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + + // Use http compression + try (Client client = newClient().useHttpCompression(false).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + + // compress request + try (Client client = newClient() + .compressClientRequest(true) + .setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true") + .useHttpCompression(true).build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + } + + @Test(groups = {"integration"}) + public void testNotFoundError() { + if (isCloud()) { + return; // not needed + } + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint("http://" + node.getHost() + ":" + node.getPort() + "/some-path") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + + try (Client client = clientBuilder.build()) { + client.queryAll("select 1"); + fail("Exception expected"); + } catch (ClientException e) { + Assert.assertTrue(e.getCause().getMessage().startsWith("There is no handle /some-path?")); + } + + } + + @Test(groups = {"integration"}) + public void testSmallNetworkBufferDoesNotBreakColumnDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + Assert.assertTrue(validBody.length > 3, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - 5); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading truncated binary stream"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testChunkedResponsePrematureEndIsReported() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT 10"; + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader(HttpHeaders.TRANSFER_ENCODING, "chunked") + .withFault(Fault.MALFORMED_RESPONSE_CHUNK)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, null, 30, + "Expected failure when reading malformed chunked response"); + ConnectionClosedException connectionClosedException = findCause(thrown, ConnectionClosedException.class); + boolean hasChunkedPrematureCloseSignature = containsMessageInCauseChain(thrown, + "closing chunk expected", + "premature end of chunk coded message body", + "failed to read header"); + Assert.assertTrue(connectionClosedException != null || hasChunkedPrematureCloseSignature, + "Expected chunked/premature-close failure signature, but was: " + thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailCorruptedStreamFailsDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 100_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading binary stream truncated at tail"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailStreamFailureReportsPositiveTimeSinceLastNextCall() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize) + .build(); + QueryResponse response = client.query(query, querySettings).get(60, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + final int[] rowsRead = new int[] {0}; + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + while (true) { + if (rowsRead[0] >= 5) { + try { + Thread.sleep(25); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + if (reader.next() == null) { + return; + } + rowsRead[0]++; + } + }); + + Assert.assertTrue(rowsRead[0] >= 5, + "Expected to read at least a few rows before failure, but read " + rowsRead[0]); + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + + String elapsedTimeMessage = findFirstMessageInCauseChain(thrown, "time since last next call"); + Assert.assertNotNull(elapsedTimeMessage, + "Expected elapsed-time fragment in failure message chain, but was: " + thrown); + + java.util.regex.Matcher matcher = Pattern.compile("time since last next call (\\d+)\\)") + .matcher(elapsedTimeMessage); + Assert.assertTrue(matcher.find(), + "Expected elapsed-time fragment in message: " + elapsedTimeMessage); + long elapsedSinceLastNext = Long.parseLong(matcher.group(1)); + Assert.assertTrue(elapsedSinceLastNext > 0, + "Expected positive elapsed time since last next call, but was " + elapsedSinceLastNext); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") + public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withBody(responseBody)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected to contain '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusErrorBodyDataProvider") + public static Object[][] testHttpStatusErrorBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: invalid credentials for user default", "invalid credentials"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: user default has no access to this operation", "no access"}, + {HttpStatus.SC_NOT_FOUND, "Not found: requested endpoint does not exist", "requested endpoint"} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusWithoutBodyDataProvider") + public void testHttpStatusErrorsWithoutBody(int httpStatus) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withStatus(httpStatus)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains("unknown server error"), + "Expected unknown error message for empty body, but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusWithoutBodyDataProvider") + public static Object[][] testHttpStatusWithoutBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED}, + {HttpStatus.SC_FORBIDDEN}, + {HttpStatus.SC_NOT_FOUND} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusCompressedBodyDataProvider") + public void testHttpStatusErrorsWithHttpCompression(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withHeader(HttpHeaders.CONTENT_ENCODING, "gzip") + .withBody(gzip(responseBody))) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .useHttpCompression(true) + .compressServerResponse(true) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected compressed body part '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusCompressedBodyDataProvider") + public static Object[][] testHttpStatusCompressedBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: token is expired", "token is expired"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: policy denies this query", "policy denies"}, + {HttpStatus.SC_NOT_FOUND, "Not found: route does not exist", "route does not exist"} + }; + } + + private byte[] fetchBinaryPayload(String query, int networkBufferSize, int timeoutSec) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newClient() + .useHttpCompression(false) + .compressServerResponse(false) + .setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)) + .build(); + QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS)) { + return readAllBytes(response.getInputStream()); + } + } + + private Throwable assertBinaryDecodeFails(WireMockServer mockServer, String query, Integer networkBufferSize, + int timeoutSec, String assertionMessage) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize).build()) { + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + try (QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + readAllRows(reader); + } + }); + Assert.assertNotNull(thrown, assertionMessage); + return thrown; + } + } + + private Client.Builder newMockServerClient(int port, Integer networkBufferSize) { + Client.Builder builder = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", port, false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1") + .useHttpCompression(false) + .compressServerResponse(false); + if (networkBufferSize != null) { + builder.setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)); + } + return builder; + } + + private static void readAllRows(ClickHouseBinaryFormatReader reader) { + while (reader.next() != null) { + reader.getInteger(1); + reader.getLong(2); + reader.getString(3); + } + } + + private static byte[] gzip(String value) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { + gzipOutputStream.write(value.getBytes(StandardCharsets.UTF_8)); + } + return out.toByteArray(); + } + + private static byte[] readAllBytes(java.io.InputStream inputStream) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + byte[] buffer = new byte[8192]; + int read; + while ((read = inputStream.read(buffer)) != -1) { + out.write(buffer, 0, read); + } + return out.toByteArray(); + } + + private static ServerException findServerException(Throwable throwable) { + Throwable current = throwable; + while (current != null) { + if (current instanceof ServerException) { + return (ServerException) current; + } + current = current.getCause(); + } + return null; + } + + private static T findCause(Throwable throwable, Class clazz) { + Throwable current = throwable; + while (current != null) { + if (clazz.isInstance(current)) { + return clazz.cast(current); + } + current = current.getCause(); + } + return null; + } + + private static boolean containsMessageInCauseChain(Throwable throwable, String... parts) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null) { + String lower = message.toLowerCase(); + for (String part : parts) { + if (lower.contains(part.toLowerCase())) { + return true; + } + } + } + current = current.getCause(); + } + return false; + } + + private static String findFirstMessageInCauseChain(Throwable throwable, String part) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null && message.contains(part)) { + return message; + } + current = current.getCause(); + } + return null; + } + + private static void assertBinaryReadFailureContainsColumnName(Throwable thrown) { + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + } + + protected Client.Builder newClient() { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + boolean isSecure = isCloud(); + return new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + } +} From c2de1fa97ebd01f37440fd81121c6e19da228b55 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 2 Apr 2026 15:15:41 -0700 Subject: [PATCH 5/6] fixed reading error message from compressed --- .../api/internal/HttpAPIClientHelper.java | 2128 +++++----- .../client/api/internal/LZ4Entity.java | 1 + .../clickhouse/client/HttpTransportTests.java | 3466 ++++++++--------- 3 files changed, 2797 insertions(+), 2798 deletions(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java index ddc41d798..24cb82aa5 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/HttpAPIClientHelper.java @@ -1,1062 +1,1066 @@ -package com.clickhouse.client.api.internal; - -import com.clickhouse.client.ClickHouseSslContextProvider; -import com.clickhouse.client.api.ClickHouseException; -import com.clickhouse.client.api.Client; -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.ClientFaultCause; -import com.clickhouse.client.api.ClientMisconfigurationException; -import com.clickhouse.client.api.ConnectionInitiationException; -import com.clickhouse.client.api.ConnectionReuseStrategy; -import com.clickhouse.client.api.DataTransferException; -import com.clickhouse.client.api.ServerException; -import com.clickhouse.client.api.enums.ProxyType; -import com.clickhouse.client.api.http.ClickHouseHttpProto; -import com.clickhouse.client.api.transport.Endpoint; -import com.clickhouse.data.ClickHouseFormat; -import net.jpountz.lz4.LZ4Factory; -import org.apache.commons.compress.compressors.CompressorStreamFactory; -import org.apache.hc.client5.http.ConnectTimeoutException; -import org.apache.hc.client5.http.classic.methods.HttpPost; -import org.apache.hc.client5.http.config.ConnectionConfig; -import org.apache.hc.client5.http.config.RequestConfig; -import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder; -import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.client5.http.impl.io.BasicHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; -import org.apache.hc.client5.http.io.HttpClientConnectionManager; -import org.apache.hc.client5.http.io.ManagedHttpClientConnection; -import org.apache.hc.client5.http.protocol.HttpClientContext; -import org.apache.hc.client5.http.socket.ConnectionSocketFactory; -import org.apache.hc.client5.http.socket.LayeredConnectionSocketFactory; -import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory; -import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; -import org.apache.hc.core5.http.ClassicHttpResponse; -import org.apache.hc.core5.http.ConnectionRequestTimeoutException; -import org.apache.hc.core5.http.ContentType; -import org.apache.hc.core5.http.Header; -import org.apache.hc.core5.http.HttpEntity; -import org.apache.hc.core5.http.HttpHeaders; -import org.apache.hc.core5.http.HttpHost; -import org.apache.hc.core5.http.HttpRequest; -import org.apache.hc.core5.http.HttpStatus; -import org.apache.hc.core5.http.NoHttpResponseException; -import org.apache.hc.core5.http.config.CharCodingConfig; -import org.apache.hc.core5.http.config.Http1Config; -import org.apache.hc.core5.http.config.RegistryBuilder; -import org.apache.hc.core5.http.impl.io.DefaultHttpResponseParserFactory; -import org.apache.hc.core5.http.io.SocketConfig; -import org.apache.hc.core5.http.io.entity.ByteArrayEntity; -import org.apache.hc.core5.http.io.entity.EntityTemplate; -import org.apache.hc.core5.http.protocol.HttpContext; -import org.apache.hc.core5.io.CloseMode; -import org.apache.hc.core5.io.IOCallback; -import org.apache.hc.core5.net.URIBuilder; -import org.apache.hc.core5.pool.ConnPoolControl; -import org.apache.hc.core5.pool.PoolConcurrencyPolicy; -import org.apache.hc.core5.pool.PoolReusePolicy; -import org.apache.hc.core5.util.TimeValue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.net.ssl.HostnameVerifier; -import javax.net.ssl.SNIHostName; -import javax.net.ssl.SSLContext; -import javax.net.ssl.SSLException; -import javax.net.ssl.SSLParameters; -import javax.net.ssl.SSLSocket; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UnsupportedEncodingException; -import java.lang.reflect.Method; -import java.net.ConnectException; -import java.net.InetSocketAddress; -import java.net.NoRouteToHostException; -import java.net.Socket; -import java.net.SocketTimeoutException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.net.UnknownHostException; -import java.nio.charset.StandardCharsets; -import java.security.NoSuchAlgorithmException; -import java.util.Arrays; -import java.util.Base64; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.regex.Pattern; -import java.util.stream.Stream; - -public class HttpAPIClientHelper { - - public static final String KEY_STATEMENT_PARAMS = "statement_params"; - - private static final Logger LOG = LoggerFactory.getLogger(HttpAPIClientHelper.class); - - private static final int ERROR_BODY_BUFFER_SIZE = 1024; // Error messages are usually small - - private final String DEFAULT_HTTP_COMPRESSION_ALGO = "lz4"; - - private static final Pattern PATTERN_HEADER_VALUE_ASCII = Pattern.compile( - "\\p{Graph}+(?:[ ]\\p{Graph}+)*"); - - private final CloseableHttpClient httpClient; - - private String proxyAuthHeaderValue; - - private final Set defaultRetryCauses; - - private final String defaultUserAgent; - private final Object metricsRegistry; - - ConnPoolControl poolControl; - - LZ4Factory lz4Factory; - - public HttpAPIClientHelper(Map configuration, Object metricsRegistry, boolean initSslContext, LZ4Factory lz4Factory) { - this.metricsRegistry = metricsRegistry; - this.httpClient = createHttpClient(initSslContext, configuration); - this.lz4Factory = lz4Factory; - assert this.lz4Factory != null; - - boolean usingClientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration); - boolean usingServerCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(configuration); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(configuration); - - LOG.debug("client compression: {}, server compression: {}, http compression: {}", usingClientCompression, usingServerCompression, useHttpCompression); - - defaultRetryCauses = new HashSet<>(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(configuration)); - if (defaultRetryCauses.contains(ClientFaultCause.None)) { - defaultRetryCauses.removeIf(c -> c != ClientFaultCause.None); - } - - this.defaultUserAgent = buildDefaultUserAgent(); - } - - /** - * Creates or returns default SSL context. - * - * @return SSLContext - */ - public SSLContext createSSLContext(Map configuration) { - SSLContext sslContext; - try { - sslContext = SSLContext.getDefault(); - } catch (NoSuchAlgorithmException e) { - throw new ClientException("Failed to create default SSL context", e); - } - ClickHouseSslContextProvider sslContextProvider = ClickHouseSslContextProvider.getProvider(); - String trustStorePath = (String) configuration.get(ClientConfigProperties.SSL_TRUST_STORE.getKey()); - if (trustStorePath != null) { - try { - sslContext = sslContextProvider.getSslContextFromKeyStore( - trustStorePath, - (String) configuration.get(ClientConfigProperties.SSL_KEY_STORE_PASSWORD.getKey()), - (String) configuration.get(ClientConfigProperties.SSL_KEYSTORE_TYPE.getKey()) - ); - } catch (SSLException e) { - throw new ClientMisconfigurationException("Failed to create SSL context from a keystore", e); - } - } else if (configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) != null || - configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()) != null || - configuration.get(ClientConfigProperties.SSL_KEY.getKey()) != null) { - - try { - sslContext = sslContextProvider.getSslContextFromCerts( - (String) configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()), - (String) configuration.get(ClientConfigProperties.SSL_KEY.getKey()), - (String) configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) - ); - } catch (SSLException e) { - throw new ClientMisconfigurationException("Failed to create SSL context from certificates", e); - } - } - return sslContext; - } - - private static final long CONNECTION_INACTIVITY_CHECK = 5000L; - - private ConnectionConfig createConnectionConfig(Map configuration) { - ConnectionConfig.Builder connConfig = ConnectionConfig.custom(); - - ClientConfigProperties.CONNECTION_TTL.applyIfSet(configuration, (t) -> connConfig.setTimeToLive(t, TimeUnit.MILLISECONDS)); - ClientConfigProperties.CONNECTION_TIMEOUT.applyIfSet(configuration, (t) -> connConfig.setConnectTimeout(t, TimeUnit.MILLISECONDS)); - connConfig.setValidateAfterInactivity(CONNECTION_INACTIVITY_CHECK, TimeUnit.MILLISECONDS); // non-configurable for now - - return connConfig.build(); - } - - private HttpClientConnectionManager basicConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { - RegistryBuilder registryBuilder = RegistryBuilder.create(); - registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory()); - registryBuilder.register("https", sslConnectionSocketFactory); - - BasicHttpClientConnectionManager connManager = new BasicHttpClientConnectionManager(registryBuilder.build()); - connManager.setConnectionConfig(createConnectionConfig(configuration)); - connManager.setSocketConfig(socketConfig); - - return connManager; - } - - private HttpClientConnectionManager poolConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { - PoolingHttpClientConnectionManagerBuilder connMgrBuilder = PoolingHttpClientConnectionManagerBuilder.create() - .setPoolConcurrencyPolicy(PoolConcurrencyPolicy.LAX); - - ConnectionReuseStrategy connectionReuseStrategy = ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getOrDefault(configuration); - switch (connectionReuseStrategy) { - case LIFO: - connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.LIFO); - break; - case FIFO: - connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.FIFO); - break; - default: - throw new ClientMisconfigurationException("Unknown connection reuse strategy: " + connectionReuseStrategy); - } - LOG.debug("Connection reuse strategy: {}", connectionReuseStrategy); - - connMgrBuilder.setDefaultConnectionConfig(createConnectionConfig(configuration)); - connMgrBuilder.setMaxConnTotal(Integer.MAX_VALUE); // as we do not know how many routes we will have - ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.applyIfSet(configuration, connMgrBuilder::setMaxConnPerRoute); - - int networkBufferSize = ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getOrDefault(configuration); - MeteredManagedHttpClientConnectionFactory connectionFactory = new MeteredManagedHttpClientConnectionFactory( - Http1Config.custom() - .setBufferSize(networkBufferSize) - .build(), - CharCodingConfig.DEFAULT, - DefaultHttpResponseParserFactory.INSTANCE); - - connMgrBuilder.setConnectionFactory(connectionFactory); - connMgrBuilder.setSSLSocketFactory(sslConnectionSocketFactory); - connMgrBuilder.setDefaultSocketConfig(socketConfig); - PoolingHttpClientConnectionManager phccm = connMgrBuilder.build(); - poolControl = phccm; - if (metricsRegistry != null) { - try { - String mGroupName = ClientConfigProperties.METRICS_GROUP_NAME.getOrDefault(configuration); - Class micrometerLoader = getClass().getClassLoader().loadClass("com.clickhouse.client.api.metrics.MicrometerLoader"); - Method applyMethod = micrometerLoader.getDeclaredMethod("applyPoolingMetricsBinder", Object.class, String.class, PoolingHttpClientConnectionManager.class); - applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, phccm); - - applyMethod = micrometerLoader.getDeclaredMethod("applyConnectionMetricsBinder", Object.class, String.class, MeteredManagedHttpClientConnectionFactory.class); - applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, connectionFactory); - } catch (Exception e) { - LOG.error("Failed to register metrics", e); - } - } - return phccm; - } - - public CloseableHttpClient createHttpClient(boolean initSslContext, Map configuration) { - // Top Level builders - HttpClientBuilder clientBuilder = HttpClientBuilder.create(); - SSLContext sslContext = initSslContext ? createSSLContext(configuration) : null; - LayeredConnectionSocketFactory sslConnectionSocketFactory; - if (sslContext != null) { - String socketSNI = (String)configuration.get(ClientConfigProperties.SSL_SOCKET_SNI.getKey()); - if (socketSNI != null && !socketSNI.trim().isEmpty()) { - sslConnectionSocketFactory = new CustomSSLConnectionFactory(socketSNI, sslContext, (hostname, session) -> true); - } else { - sslConnectionSocketFactory = new SSLConnectionSocketFactory(sslContext); - } - } else { - sslConnectionSocketFactory = new DummySSLConnectionSocketFactory(); - } - // Socket configuration - SocketConfig.Builder soCfgBuilder = SocketConfig.custom(); - ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.applyIfSet(configuration, - (t) -> soCfgBuilder.setSoTimeout(t, TimeUnit.MILLISECONDS)); - - ClientConfigProperties.SOCKET_RCVBUF_OPT.applyIfSet(configuration, - soCfgBuilder::setRcvBufSize); - - ClientConfigProperties.SOCKET_SNDBUF_OPT.applyIfSet(configuration, - soCfgBuilder::setSndBufSize); - - ClientConfigProperties.SOCKET_LINGER_OPT.applyIfSet(configuration, - (v) -> soCfgBuilder.setSoLinger(v, TimeUnit.SECONDS)); - - ClientConfigProperties.SOCKET_TCP_NO_DELAY_OPT.applyIfSet(configuration, - soCfgBuilder::setTcpNoDelay); - // Proxy - String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey()); - Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey()); - HttpHost proxy = null; - if (proxyHost != null && proxyPort != null) { - proxy = new HttpHost(proxyHost, proxyPort); - } - - String proxyTypeVal = (String) configuration.get(ClientConfigProperties.PROXY_TYPE.getKey()); - ProxyType proxyType = proxyTypeVal == null ? null : ProxyType.valueOf(proxyTypeVal); - if (proxyType == ProxyType.HTTP) { - clientBuilder.setProxy(proxy); - String proxyUser = (String) configuration.get(ClientConfigProperties.PROXY_USER.getKey()); - String proxyPassword = (String) configuration.get(ClientConfigProperties.PROXY_PASSWORD.getKey()); - if (proxyUser != null && proxyPassword != null) { - proxyAuthHeaderValue = "Basic " + Base64.getEncoder().encodeToString((proxyUser + ":" + proxyPassword).getBytes(StandardCharsets.UTF_8)); - } - - } else if (proxyType == ProxyType.SOCKS) { - soCfgBuilder.setSocksProxyAddress(new InetSocketAddress(proxyHost, proxyPort)); - } - - boolean disableCookies = !((Boolean)ClientConfigProperties.HTTP_SAVE_COOKIES.getOrDefault(configuration)); - if (disableCookies) { - clientBuilder.disableCookieManagement(); - } - SocketConfig socketConfig = soCfgBuilder.build(); - - // Connection manager - if (ClientConfigProperties.CONNECTION_POOL_ENABLED.getOrDefault(configuration)) { - clientBuilder.setConnectionManager(poolConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); - } else { - clientBuilder.setConnectionManager(basicConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); - } - Long keepAliveTimeout = ClientConfigProperties.HTTP_KEEP_ALIVE_TIMEOUT.getOrDefault(configuration); - if (keepAliveTimeout != null && keepAliveTimeout > 0) { - clientBuilder.setKeepAliveStrategy((response, context) -> TimeValue.ofMilliseconds(keepAliveTimeout)); - } - - clientBuilder.disableContentCompression(); // will handle ourselves - - return clientBuilder.build(); - } - - private static final String ERROR_CODE_PREFIX_PATTERN = "%d. DB::Exception:"; - - /** - * Reads status line and if error tries to parse response body to get server error message. - * - * @param httpResponse - HTTP response - * @return exception object with server code - */ - public Exception readError(HttpPost req, ClassicHttpResponse httpResponse) { - final Header serverQueryIdHeader = httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); - final Header clientQueryIdHeader = req.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); - final Header queryHeader = Stream.of(serverQueryIdHeader, clientQueryIdHeader).filter(Objects::nonNull).findFirst().orElse(null); - final String queryId = queryHeader == null ? "" : queryHeader.getValue(); - int serverCode = getHeaderInt(httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE), 0); - try { - return serverCode > 0 ? readClickHouseError(httpResponse.getEntity(), serverCode, queryId, httpResponse.getCode()) : - readNotClickHouseError(httpResponse.getEntity(), queryId, httpResponse.getCode()); - } catch (Exception e) { - LOG.error("Failed to read error message", e); - String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; - return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); - } - } - - private ServerException readNotClickHouseError(HttpEntity httpEntity, String queryId, int httpCode) { - - byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; - - String msg = null; - InputStream body = null; - for (int i = 0; i < 2; i++) { - try { - if (body == null) { - body = httpEntity.getContent(); - } - int msgLen = body.read(buffer, 0, buffer.length - 2); - if (msgLen > 0) { - msg = new String(buffer, 0, msgLen, StandardCharsets.UTF_8).trim(); - if (msg.isEmpty()) { - msg = ""; - } - } - break; - } catch (ClientException e) { - // Invalid LZ4 Magic - if (body instanceof ClickHouseLZ4InputStream) { - ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; - body = stream.getInputStream(); - continue; - } - throw e; - } catch (Exception e) { - LOG.warn("Failed to read error message (queryId = " + queryId + ")", e); - break; - } - } - - String errormsg = msg == null ? "unknown server error" : msg; - return new ServerException(ServerException.CODE_UNKNOWN, errormsg + " (transport error: " + httpCode +")", httpCode, queryId); - } - - private static ServerException readClickHouseError(HttpEntity httpEntity, int serverCode, String queryId, int httpCode) throws Exception { - InputStream body = httpEntity.getContent(); - byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; - byte[] lookUpStr = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode).getBytes(StandardCharsets.UTF_8); - StringBuilder msgBuilder = new StringBuilder(); - boolean found = false; - while (true) { - int rBytes = -1; - try { - rBytes = body.read(buffer); - } catch (ClientException e) { - // Invalid LZ4 Magic - if (body instanceof ClickHouseLZ4InputStream) { - ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; - body = stream.getInputStream(); - byte[] headerBuffer = stream.getHeaderBuffer(); - System.arraycopy(headerBuffer, 0, buffer, 0, headerBuffer.length); - rBytes = headerBuffer.length; - } - } - if (rBytes == -1) { - break; - } - - for (int i = 0; i < rBytes; i++) { - if (buffer[i] == lookUpStr[0]) { - found = true; - for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { - if (buffer[i + j] != lookUpStr[j]) { - found = false; - break; - } - } - if (found) { - msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); - break; - } - } - } - - if (found) { - break; - } - } - - while (true) { - int rBytes = body.read(buffer); - if (rBytes == -1) { - break; - } - msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); - } - - String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") - .replaceAll("\\\\/", "/"); - if (msg.trim().isEmpty()) { - msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpCode + ")"; - } - return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpCode, queryId); - } - - private static final long POOL_VENT_TIMEOUT = 10000L; - private final AtomicLong timeToPoolVent = new AtomicLong(0); - - private void doPoolVent() { - if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) { - timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT); - poolControl.closeExpired(); - } - } - - private HttpContext createRequestHttpContext(Map requestConfig) { - HttpClientContext context = HttpClientContext.create(); - Number responseTimeout = ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getOrDefault(requestConfig); - Number connectionReqTimeout = ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getOrDefault(requestConfig); - RequestConfig reqHttpConf = RequestConfig.custom() - .setResponseTimeout(responseTimeout.longValue(), TimeUnit.MILLISECONDS) - .setConnectionRequestTimeout(connectionReqTimeout.longValue(), TimeUnit.MILLISECONDS) - .build(); - context.setRequestConfig(reqHttpConf); - - return context; - } - - private URI createRequestURI(Endpoint server, Map requestConfig, boolean addParameters) { - URI uri; - try { - URIBuilder uriBuilder = new URIBuilder(server.getURI()); - addRequestParams(requestConfig, uriBuilder::addParameter); - - if (addParameters) { - addStatementParams(requestConfig, uriBuilder::addParameter); - } - - uri = uriBuilder.optimize().build(); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - return uri; - } - - private HttpPost createPostRequest(URI uri, Map requestConfig) { - HttpPost req = new HttpPost(uri); -// req.setVersion(new ProtocolVersion("HTTP", 1, 0)); // to disable chunk transfer encoding - addHeaders(req, requestConfig); - return req; - } - - public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, - String body) throws Exception { - - final URI uri = createRequestURI(server, requestConfig, true); - final HttpPost req = createPostRequest(uri, requestConfig); - final String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; - - HttpEntity httpEntity = new ByteArrayEntity(body.getBytes(StandardCharsets.UTF_8.name()), CONTENT_TYPE, contentEncoding); - req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); - - return doPostRequest(requestConfig, req); - } - - public ClassicHttpResponse executeMultiPartRequest(Endpoint server, Map requestConfig, String sqlQuery) throws Exception { - - requestConfig.put(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey(), false); - - final URI uri = createRequestURI(server, requestConfig, false); - final HttpPost req = createPostRequest(uri, requestConfig); - - MultipartEntityBuilder multipartEntityBuilder = MultipartEntityBuilder.create(); - addStatementParams(requestConfig, multipartEntityBuilder::addTextBody); - multipartEntityBuilder.addTextBody(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlQuery); - - - HttpEntity httpEntity = multipartEntityBuilder.build(); - req.setHeader(HttpHeaders.CONTENT_TYPE, httpEntity.getContentType()); // set proper content type with generated boundary value - req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); - - return doPostRequest(requestConfig, req); - - - } - - public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, - IOCallback writeCallback) throws Exception { - - final URI uri = createRequestURI(server, requestConfig, true); - final HttpPost req = createPostRequest(uri, requestConfig); - String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; - req.setEntity(wrapRequestEntity( - new EntityTemplate(-1, CONTENT_TYPE, contentEncoding , writeCallback), - requestConfig)); - - return doPostRequest(requestConfig, req); - } - - private ClassicHttpResponse doPostRequest(Map requestConfig, HttpPost req) throws Exception { - - doPoolVent(); - - ClassicHttpResponse httpResponse = null; - HttpContext context = createRequestHttpContext(requestConfig); - try { - httpResponse = httpClient.executeOpen(null, req, context); - - httpResponse.setEntity(wrapResponseEntity(httpResponse.getEntity(), - httpResponse.getCode(), - requestConfig)); - - if (httpResponse.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) { - throw new ClientMisconfigurationException("Proxy authentication required. Please check your proxy settings."); - } else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) { - httpResponse.close(); - throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings."); - } else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) { - try { - throw readError(req, httpResponse); - } finally { - httpResponse.close(); - } - } - return httpResponse; - - } catch (UnknownHostException e) { - closeQuietly(httpResponse); - LOG.warn("Host '{}' unknown", req.getAuthority()); - throw e; - } catch (ConnectException | NoRouteToHostException e) { - closeQuietly(httpResponse); - LOG.warn("Failed to connect to '{}': {}", req.getAuthority(), e.getMessage()); - throw e; - } catch (Exception e) { - closeQuietly(httpResponse); - LOG.debug("Failed to execute request to '{}': {}", req.getAuthority(), e.getMessage(), e); - throw e; - } - } - - public static void closeQuietly(ClassicHttpResponse httpResponse) { - if (httpResponse != null) { - try { - httpResponse.close(); - } catch (IOException e) { - LOG.warn("Failed to close response"); - } - } - } - - private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8"); - - private void addHeaders(HttpPost req, Map requestConfig) { - setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType()); - if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) { - setHeader( - req, - ClickHouseHttpProto.HEADER_FORMAT, - ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name()); - } - if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { - setHeader( - req, - ClickHouseHttpProto.HEADER_QUERY_ID, - (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey())); - } - setHeader( - req, - ClickHouseHttpProto.HEADER_DATABASE, - ClientConfigProperties.DATABASE.getOrDefault(requestConfig)); - - if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) { - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_USER, - ClientConfigProperties.USER.getOrDefault(requestConfig)); - setHeader( - req, - ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, - "on"); - } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) { - String user = ClientConfigProperties.USER.getOrDefault(requestConfig); - String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig); - if (password == null) { - password = ""; - } - // Use as-is, no encoding allowed - req.addHeader( - HttpHeaders.AUTHORIZATION, - "Basic " + Base64.getEncoder().encodeToString( - (user + ":" + password).getBytes(StandardCharsets.UTF_8))); - } else { - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_USER, - ClientConfigProperties.USER.getOrDefault(requestConfig)); - setHeader( - req, - ClickHouseHttpProto.HEADER_DB_PASSWORD, - ClientConfigProperties.PASSWORD.getOrDefault(requestConfig)); - } - if (proxyAuthHeaderValue != null) { - req.addHeader( - HttpHeaders.PROXY_AUTHORIZATION, - proxyAuthHeaderValue); - } - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); - - if (useHttpCompression) { - if (serverCompression) { - setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); - } - - if (clientCompression && !appCompressedData) { - setHeader(req, HttpHeaders.CONTENT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); - } - } - - for (String key : requestConfig.keySet()) { - if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { - Object val = requestConfig.get(key); - if (val != null) { - setHeader( - req, - key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), - String.valueOf(val)); - } - } - } - - // Special cases - if (req.containsHeader(HttpHeaders.AUTHORIZATION) - && (req.containsHeader(ClickHouseHttpProto.HEADER_DB_USER) || - req.containsHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD))) - { - // user has set auth header for purpose, lets remove ours - req.removeHeaders(ClickHouseHttpProto.HEADER_DB_USER); - req.removeHeaders(ClickHouseHttpProto.HEADER_DB_PASSWORD); - } - - // -- keep last - correctUserAgentHeader(req, requestConfig); - } - - private void addRequestParams(Map requestConfig, BiConsumer consumer) { - if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { - consumer.accept(ClickHouseHttpProto.QPARAM_QUERY_ID, requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()).toString()); - } - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - - if (useHttpCompression) { - // enable_http_compression make server react on http header - // for client side compression Content-Encoding should be set - // for server side compression Accept-Encoding should be set - consumer.accept("enable_http_compression", "1"); - } else { - if (serverCompression) { - consumer.accept("compress", "1"); - } - if (clientCompression) { - consumer.accept("decompress", "1"); - } - } - - Collection sessionRoles = ClientConfigProperties.SESSION_DB_ROLES.getOrDefault(requestConfig); - if (!(sessionRoles == null || sessionRoles.isEmpty())) { - sessionRoles.forEach(r -> consumer.accept(ClickHouseHttpProto.QPARAM_ROLE, r)); - } - - for (String key : requestConfig.keySet()) { - if (key.startsWith(ClientConfigProperties.SERVER_SETTING_PREFIX)) { - Object val = requestConfig.get(key); - if (val != null) { - consumer.accept(key.substring(ClientConfigProperties.SERVER_SETTING_PREFIX.length()), String.valueOf(requestConfig.get(key))); - } - } - } - } - - private void addStatementParams(Map requestConfig, BiConsumer consumer) { - if (requestConfig.containsKey(KEY_STATEMENT_PARAMS)) { - Map params = (Map) requestConfig.get(KEY_STATEMENT_PARAMS); - params.forEach((k, v) -> consumer.accept("param_" + k, String.valueOf(v))); - } - } - - private HttpEntity wrapRequestEntity(HttpEntity httpEntity, Map requestConfig) { - - boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); - - LOG.debug("wrapRequestEntity: client compression: {}, http compression: {}, content encoding: {}", - clientCompression, useHttpCompression, httpEntity.getContentEncoding()); - - if (httpEntity.getContentEncoding() != null && !appCompressedData) { - // http header is set and data is not compressed - return new CompressedEntity(httpEntity, false, CompressorStreamFactory.getSingleton()); - } else if (clientCompression && !appCompressedData) { - int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); - return new LZ4Entity(httpEntity, useHttpCompression, false, true, - buffSize, false, lz4Factory); - } else { - return httpEntity; - } - } - - private HttpEntity wrapResponseEntity(HttpEntity httpEntity, int httpStatus, Map requestConfig) { - boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); - boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); - - LOG.debug("wrapResponseEntity: server compression: {}, http compression: {}, content encoding: {}", - serverCompression, useHttpCompression, httpEntity.getContentEncoding()); - - if (httpEntity.getContentEncoding() != null) { - // http compressed response - return new CompressedEntity(httpEntity, true, CompressorStreamFactory.getSingleton()); - } - - // data compression - if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED || httpStatus == HttpStatus.SC_NOT_FOUND)) { - int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); - return new LZ4Entity(httpEntity, useHttpCompression, true, false, buffSize, true, lz4Factory); - } - - return httpEntity; - } - - public static int getHeaderInt(Header header, int defaultValue) { - return getHeaderVal(header, defaultValue, Integer::parseInt); - } - - private static final Set RESPONSE_HEADER_WHITELIST = new HashSet<>(Arrays.asList( - ClickHouseHttpProto.HEADER_QUERY_ID, - ClickHouseHttpProto.HEADER_SRV_SUMMARY, - ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME, - ClickHouseHttpProto.HEADER_DATABASE, - ClickHouseHttpProto.HEADER_DB_USER - )); - - /** - * Collects whitelisted response headers from an HTTP response into a map. - * - * @param response the HTTP response - * @return unmodifiable map of header name to header value for whitelisted headers present in the response - */ - public static Map collectResponseHeaders(ClassicHttpResponse response) { - Map headers = new HashMap<>(); - for (String name : RESPONSE_HEADER_WHITELIST) { - Header header = response.getFirstHeader(name); - if (header != null) { - headers.put(name, header.getValue()); - } - } - return Collections.unmodifiableMap(headers); - } - - public static String getHeaderVal(Header header, String defaultValue) { - return getHeaderVal(header, defaultValue, Function.identity()); - } - - public static T getHeaderVal(Header header, T defaultValue, Function converter) { - if (header == null) { - return defaultValue; - } - - return converter.apply(header.getValue()); - } - - public boolean shouldRetry(Throwable ex, Map requestSettings) { - List retryCauses = ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(requestSettings); - - if (retryCauses.contains(ClientFaultCause.None)) { - return false; - } - - if (ex instanceof NoHttpResponseException - || ex.getCause() instanceof NoHttpResponseException) { - return retryCauses.contains(ClientFaultCause.NoHttpResponse); - } - - if (ex instanceof ConnectException - || ex instanceof ConnectTimeoutException - || ex.getCause() instanceof ConnectException - || ex.getCause() instanceof ConnectTimeoutException) { - return retryCauses.contains(ClientFaultCause.ConnectTimeout); - } - - if (ex instanceof ConnectionRequestTimeoutException - || ex.getCause() instanceof ConnectionRequestTimeoutException) { - return retryCauses.contains(ClientFaultCause.ConnectionRequestTimeout); - } - - if (ex instanceof SocketTimeoutException - || ex.getCause() instanceof SocketTimeoutException) { - return retryCauses.contains(ClientFaultCause.SocketTimeout); - } - - // there are some db retryable error codes - if (ex instanceof ServerException || ex.getCause() instanceof ServerException) { - ServerException se = (ServerException) ex; - return se.isRetryable() && retryCauses.contains(ClientFaultCause.ServerRetryable); - } - - return false; - } - - // This method wraps some client specific exceptions into specific ClientException or just ClientException - // ClientException will be also wrapped - public RuntimeException wrapException(String message, Exception cause, String queryId) { - if (cause instanceof ClientException || cause instanceof ServerException) { - return (RuntimeException) cause; - } - - if (cause instanceof ConnectionRequestTimeoutException || - cause instanceof NoHttpResponseException || - cause instanceof ConnectTimeoutException || - cause instanceof ConnectException || - cause instanceof UnknownHostException || - cause instanceof NoRouteToHostException) { - ConnectionInitiationException ex = new ConnectionInitiationException(message, cause); - ex.setQueryId(queryId); - return ex; - } - - if (cause instanceof SocketTimeoutException || cause instanceof IOException) { - DataTransferException ex = new DataTransferException(message, cause); - ex.setQueryId(queryId); - return ex; - } - // if we can not identify the exception explicitly we catch as our base exception ClickHouseException - return new ClickHouseException(message, cause, queryId); - } - - private void correctUserAgentHeader(HttpRequest request, Map requestConfig) { - //TODO: implement cache for user-agent - Header userAgentHeader = request.getLastHeader(HttpHeaders.USER_AGENT); - request.removeHeaders(HttpHeaders.USER_AGENT); - - String clientName = ClientConfigProperties.CLIENT_NAME.getOrDefault(requestConfig); - String userAgentValue = defaultUserAgent; - if (userAgentHeader == null && clientName != null && !clientName.isEmpty()) { - userAgentValue = clientName + " " + defaultUserAgent; - } else if (userAgentHeader != null) { - userAgentValue = userAgentHeader.getValue() + " " + defaultUserAgent; - } - request.setHeader(HttpHeaders.USER_AGENT, userAgentValue); - } - - private String buildDefaultUserAgent() { - StringBuilder userAgent = new StringBuilder(); - userAgent.append(Client.CLIENT_USER_AGENT); - - String clientVersion = Client.clientVersion; - - userAgent.append(clientVersion); - - userAgent.append(" ("); - userAgent.append(System.getProperty("os.name")); - userAgent.append("; "); - userAgent.append("jvm:").append(System.getProperty("java.version")); - userAgent.append("; "); - - userAgent.setLength(userAgent.length() - 2); - userAgent.append(')'); - - try { - String httpClientVersion = this.httpClient.getClass().getPackage().getImplementationVersion(); - if (Objects.equals(this.httpClient.getClass().getPackage().getImplementationTitle(), this.getClass().getPackage().getImplementationTitle())) { - // shaded jar - all packages have same implementation title - httpClientVersion = "unknown"; - try (InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream("client-v2-version.properties")) { - Properties p = new Properties(); - p.load(in); - - String tmp = p.getProperty("apache.http.client.version"); - if (tmp != null && !tmp.isEmpty() && !tmp.equals("${apache.httpclient.version}")) { - httpClientVersion = tmp; - } - } catch (Exception e) { - // ignore - } - } - userAgent.append(" ") - .append("Apache-HttpClient") - .append('/') - .append(httpClientVersion); - } catch (Exception e) { - LOG.info("failed to construct http client version string"); - } - return userAgent.toString(); - } - - public void close() { - httpClient.close(CloseMode.IMMEDIATE); - } - - private static void setHeader(HttpRequest req, String headerName, - String value) - { - if (value == null) { - return; - } - - if (value.trim().isEmpty()) { - return; - } - if (PATTERN_HEADER_VALUE_ASCII.matcher(value).matches()) { - req.setHeader(headerName, value); - } else { - try { - req.setHeader( - headerName + "*", - "UTF-8''" + URLEncoder.encode(value, StandardCharsets.UTF_8.name())); - } catch (UnsupportedEncodingException e) { - throw new ClientException("Failed to convert string to UTF8" , e); - } - } - } - - /** - * This factory is used only when no ssl connections are required (no https endpoints). - * Internally http client would create factory and spend time if no supplied. - */ - private static class DummySSLConnectionSocketFactory implements LayeredConnectionSocketFactory { - @Override - public Socket createLayeredSocket(Socket socket, String target, int port, HttpContext context) throws IOException { - return null; - } - - @Override - public Socket createSocket(HttpContext context) throws IOException { - return null; - } - - @Override - public Socket connectSocket(TimeValue connectTimeout, Socket socket, HttpHost host, InetSocketAddress remoteAddress, InetSocketAddress localAddress, HttpContext context) throws IOException { - return null; - } - } - - public static class MeteredManagedHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory { - public MeteredManagedHttpClientConnectionFactory(Http1Config http1Config, CharCodingConfig charCodingConfig, DefaultHttpResponseParserFactory defaultHttpResponseParserFactory) { - super(http1Config, charCodingConfig, defaultHttpResponseParserFactory); - } - - ConcurrentLinkedQueue times = new ConcurrentLinkedQueue<>(); - - - @Override - public ManagedHttpClientConnection createConnection(Socket socket) throws IOException { - long startT = System.currentTimeMillis(); - try { - return super.createConnection(socket); - } finally { - long endT = System.currentTimeMillis(); - times.add(endT - startT); - } - } - - public long getTime() { - int count = times.size(); - long runningAverage = 0; - for (int i = 0; i < count; i++) { - Long t = times.poll(); - if (t != null) { - runningAverage += t; - } - } - - return count > 0 ? runningAverage / count : 0; - } - } - - public static class CustomSSLConnectionFactory extends SSLConnectionSocketFactory { - - private final SNIHostName defaultSNI; - - public CustomSSLConnectionFactory(String defaultSNI, SSLContext sslContext, HostnameVerifier hostnameVerifier) { - super(sslContext, hostnameVerifier); - this.defaultSNI = defaultSNI == null || defaultSNI.trim().isEmpty() ? null : new SNIHostName(defaultSNI); - } - - @Override - protected void prepareSocket(SSLSocket socket, HttpContext context) throws IOException { - super.prepareSocket(socket, context); - - if (defaultSNI != null) { - SSLParameters sslParams = socket.getSSLParameters(); - sslParams.setServerNames(Collections.singletonList(defaultSNI)); - socket.setSSLParameters(sslParams); - } - } - } -} +package com.clickhouse.client.api.internal; + +import com.clickhouse.client.ClickHouseSslContextProvider; +import com.clickhouse.client.api.ClickHouseException; +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.ClientFaultCause; +import com.clickhouse.client.api.ClientMisconfigurationException; +import com.clickhouse.client.api.ConnectionInitiationException; +import com.clickhouse.client.api.ConnectionReuseStrategy; +import com.clickhouse.client.api.DataTransferException; +import com.clickhouse.client.api.ServerException; +import com.clickhouse.client.api.enums.ProxyType; +import com.clickhouse.client.api.http.ClickHouseHttpProto; +import com.clickhouse.client.api.transport.Endpoint; +import com.clickhouse.data.ClickHouseFormat; +import net.jpountz.lz4.LZ4Factory; +import org.apache.commons.compress.compressors.CompressorStreamFactory; +import org.apache.hc.client5.http.ConnectTimeoutException; +import org.apache.hc.client5.http.classic.methods.HttpPost; +import org.apache.hc.client5.http.config.ConnectionConfig; +import org.apache.hc.client5.http.config.RequestConfig; +import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.impl.io.BasicHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.ManagedHttpClientConnectionFactory; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.client5.http.io.ManagedHttpClientConnection; +import org.apache.hc.client5.http.protocol.HttpClientContext; +import org.apache.hc.client5.http.socket.ConnectionSocketFactory; +import org.apache.hc.client5.http.socket.LayeredConnectionSocketFactory; +import org.apache.hc.client5.http.socket.PlainConnectionSocketFactory; +import org.apache.hc.client5.http.ssl.SSLConnectionSocketFactory; +import org.apache.hc.core5.http.ClassicHttpResponse; +import org.apache.hc.core5.http.ConnectionRequestTimeoutException; +import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.HttpEntity; +import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.HttpHost; +import org.apache.hc.core5.http.HttpRequest; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.http.NoHttpResponseException; +import org.apache.hc.core5.http.config.CharCodingConfig; +import org.apache.hc.core5.http.config.Http1Config; +import org.apache.hc.core5.http.config.RegistryBuilder; +import org.apache.hc.core5.http.impl.io.DefaultHttpResponseParserFactory; +import org.apache.hc.core5.http.io.SocketConfig; +import org.apache.hc.core5.http.io.entity.ByteArrayEntity; +import org.apache.hc.core5.http.io.entity.EntityTemplate; +import org.apache.hc.core5.http.protocol.HttpContext; +import org.apache.hc.core5.io.CloseMode; +import org.apache.hc.core5.io.IOCallback; +import org.apache.hc.core5.net.URIBuilder; +import org.apache.hc.core5.pool.ConnPoolControl; +import org.apache.hc.core5.pool.PoolConcurrencyPolicy; +import org.apache.hc.core5.pool.PoolReusePolicy; +import org.apache.hc.core5.util.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SNIHostName; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLException; +import javax.net.ssl.SSLParameters; +import javax.net.ssl.SSLSocket; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.Method; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.NoRouteToHostException; +import java.net.Socket; +import java.net.SocketTimeoutException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.net.UnknownHostException; +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Base64; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +public class HttpAPIClientHelper { + + public static final String KEY_STATEMENT_PARAMS = "statement_params"; + + private static final Logger LOG = LoggerFactory.getLogger(HttpAPIClientHelper.class); + + private static final int ERROR_BODY_BUFFER_SIZE = 1024; // Error messages are usually small + + private final String DEFAULT_HTTP_COMPRESSION_ALGO = "lz4"; + + private static final Pattern PATTERN_HEADER_VALUE_ASCII = Pattern.compile( + "\\p{Graph}+(?:[ ]\\p{Graph}+)*"); + + private final CloseableHttpClient httpClient; + + private String proxyAuthHeaderValue; + + private final Set defaultRetryCauses; + + private final String defaultUserAgent; + private final Object metricsRegistry; + + ConnPoolControl poolControl; + + LZ4Factory lz4Factory; + + public HttpAPIClientHelper(Map configuration, Object metricsRegistry, boolean initSslContext, LZ4Factory lz4Factory) { + this.metricsRegistry = metricsRegistry; + this.httpClient = createHttpClient(initSslContext, configuration); + this.lz4Factory = lz4Factory; + assert this.lz4Factory != null; + + boolean usingClientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(configuration); + boolean usingServerCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(configuration); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(configuration); + + LOG.debug("client compression: {}, server compression: {}, http compression: {}", usingClientCompression, usingServerCompression, useHttpCompression); + + defaultRetryCauses = new HashSet<>(ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(configuration)); + if (defaultRetryCauses.contains(ClientFaultCause.None)) { + defaultRetryCauses.removeIf(c -> c != ClientFaultCause.None); + } + + this.defaultUserAgent = buildDefaultUserAgent(); + } + + /** + * Creates or returns default SSL context. + * + * @return SSLContext + */ + public SSLContext createSSLContext(Map configuration) { + SSLContext sslContext; + try { + sslContext = SSLContext.getDefault(); + } catch (NoSuchAlgorithmException e) { + throw new ClientException("Failed to create default SSL context", e); + } + ClickHouseSslContextProvider sslContextProvider = ClickHouseSslContextProvider.getProvider(); + String trustStorePath = (String) configuration.get(ClientConfigProperties.SSL_TRUST_STORE.getKey()); + if (trustStorePath != null) { + try { + sslContext = sslContextProvider.getSslContextFromKeyStore( + trustStorePath, + (String) configuration.get(ClientConfigProperties.SSL_KEY_STORE_PASSWORD.getKey()), + (String) configuration.get(ClientConfigProperties.SSL_KEYSTORE_TYPE.getKey()) + ); + } catch (SSLException e) { + throw new ClientMisconfigurationException("Failed to create SSL context from a keystore", e); + } + } else if (configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) != null || + configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()) != null || + configuration.get(ClientConfigProperties.SSL_KEY.getKey()) != null) { + + try { + sslContext = sslContextProvider.getSslContextFromCerts( + (String) configuration.get(ClientConfigProperties.SSL_CERTIFICATE.getKey()), + (String) configuration.get(ClientConfigProperties.SSL_KEY.getKey()), + (String) configuration.get(ClientConfigProperties.CA_CERTIFICATE.getKey()) + ); + } catch (SSLException e) { + throw new ClientMisconfigurationException("Failed to create SSL context from certificates", e); + } + } + return sslContext; + } + + private static final long CONNECTION_INACTIVITY_CHECK = 5000L; + + private ConnectionConfig createConnectionConfig(Map configuration) { + ConnectionConfig.Builder connConfig = ConnectionConfig.custom(); + + ClientConfigProperties.CONNECTION_TTL.applyIfSet(configuration, (t) -> connConfig.setTimeToLive(t, TimeUnit.MILLISECONDS)); + ClientConfigProperties.CONNECTION_TIMEOUT.applyIfSet(configuration, (t) -> connConfig.setConnectTimeout(t, TimeUnit.MILLISECONDS)); + connConfig.setValidateAfterInactivity(CONNECTION_INACTIVITY_CHECK, TimeUnit.MILLISECONDS); // non-configurable for now + + return connConfig.build(); + } + + private HttpClientConnectionManager basicConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { + RegistryBuilder registryBuilder = RegistryBuilder.create(); + registryBuilder.register("http", PlainConnectionSocketFactory.getSocketFactory()); + registryBuilder.register("https", sslConnectionSocketFactory); + + BasicHttpClientConnectionManager connManager = new BasicHttpClientConnectionManager(registryBuilder.build()); + connManager.setConnectionConfig(createConnectionConfig(configuration)); + connManager.setSocketConfig(socketConfig); + + return connManager; + } + + private HttpClientConnectionManager poolConnectionManager(LayeredConnectionSocketFactory sslConnectionSocketFactory, SocketConfig socketConfig, Map configuration) { + PoolingHttpClientConnectionManagerBuilder connMgrBuilder = PoolingHttpClientConnectionManagerBuilder.create() + .setPoolConcurrencyPolicy(PoolConcurrencyPolicy.LAX); + + ConnectionReuseStrategy connectionReuseStrategy = ClientConfigProperties.CONNECTION_REUSE_STRATEGY.getOrDefault(configuration); + switch (connectionReuseStrategy) { + case LIFO: + connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.LIFO); + break; + case FIFO: + connMgrBuilder.setConnPoolPolicy(PoolReusePolicy.FIFO); + break; + default: + throw new ClientMisconfigurationException("Unknown connection reuse strategy: " + connectionReuseStrategy); + } + LOG.debug("Connection reuse strategy: {}", connectionReuseStrategy); + + connMgrBuilder.setDefaultConnectionConfig(createConnectionConfig(configuration)); + connMgrBuilder.setMaxConnTotal(Integer.MAX_VALUE); // as we do not know how many routes we will have + ClientConfigProperties.HTTP_MAX_OPEN_CONNECTIONS.applyIfSet(configuration, connMgrBuilder::setMaxConnPerRoute); + + int networkBufferSize = ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getOrDefault(configuration); + MeteredManagedHttpClientConnectionFactory connectionFactory = new MeteredManagedHttpClientConnectionFactory( + Http1Config.custom() + .setBufferSize(networkBufferSize) + .build(), + CharCodingConfig.DEFAULT, + DefaultHttpResponseParserFactory.INSTANCE); + + connMgrBuilder.setConnectionFactory(connectionFactory); + connMgrBuilder.setSSLSocketFactory(sslConnectionSocketFactory); + connMgrBuilder.setDefaultSocketConfig(socketConfig); + PoolingHttpClientConnectionManager phccm = connMgrBuilder.build(); + poolControl = phccm; + if (metricsRegistry != null) { + try { + String mGroupName = ClientConfigProperties.METRICS_GROUP_NAME.getOrDefault(configuration); + Class micrometerLoader = getClass().getClassLoader().loadClass("com.clickhouse.client.api.metrics.MicrometerLoader"); + Method applyMethod = micrometerLoader.getDeclaredMethod("applyPoolingMetricsBinder", Object.class, String.class, PoolingHttpClientConnectionManager.class); + applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, phccm); + + applyMethod = micrometerLoader.getDeclaredMethod("applyConnectionMetricsBinder", Object.class, String.class, MeteredManagedHttpClientConnectionFactory.class); + applyMethod.invoke(micrometerLoader, metricsRegistry, mGroupName, connectionFactory); + } catch (Exception e) { + LOG.error("Failed to register metrics", e); + } + } + return phccm; + } + + public CloseableHttpClient createHttpClient(boolean initSslContext, Map configuration) { + // Top Level builders + HttpClientBuilder clientBuilder = HttpClientBuilder.create(); + SSLContext sslContext = initSslContext ? createSSLContext(configuration) : null; + LayeredConnectionSocketFactory sslConnectionSocketFactory; + if (sslContext != null) { + String socketSNI = (String)configuration.get(ClientConfigProperties.SSL_SOCKET_SNI.getKey()); + if (socketSNI != null && !socketSNI.trim().isEmpty()) { + sslConnectionSocketFactory = new CustomSSLConnectionFactory(socketSNI, sslContext, (hostname, session) -> true); + } else { + sslConnectionSocketFactory = new SSLConnectionSocketFactory(sslContext); + } + } else { + sslConnectionSocketFactory = new DummySSLConnectionSocketFactory(); + } + // Socket configuration + SocketConfig.Builder soCfgBuilder = SocketConfig.custom(); + ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.applyIfSet(configuration, + (t) -> soCfgBuilder.setSoTimeout(t, TimeUnit.MILLISECONDS)); + + ClientConfigProperties.SOCKET_RCVBUF_OPT.applyIfSet(configuration, + soCfgBuilder::setRcvBufSize); + + ClientConfigProperties.SOCKET_SNDBUF_OPT.applyIfSet(configuration, + soCfgBuilder::setSndBufSize); + + ClientConfigProperties.SOCKET_LINGER_OPT.applyIfSet(configuration, + (v) -> soCfgBuilder.setSoLinger(v, TimeUnit.SECONDS)); + + ClientConfigProperties.SOCKET_TCP_NO_DELAY_OPT.applyIfSet(configuration, + soCfgBuilder::setTcpNoDelay); + // Proxy + String proxyHost = (String) configuration.get(ClientConfigProperties.PROXY_HOST.getKey()); + Integer proxyPort = (Integer) configuration.get(ClientConfigProperties.PROXY_PORT.getKey()); + HttpHost proxy = null; + if (proxyHost != null && proxyPort != null) { + proxy = new HttpHost(proxyHost, proxyPort); + } + + String proxyTypeVal = (String) configuration.get(ClientConfigProperties.PROXY_TYPE.getKey()); + ProxyType proxyType = proxyTypeVal == null ? null : ProxyType.valueOf(proxyTypeVal); + if (proxyType == ProxyType.HTTP) { + clientBuilder.setProxy(proxy); + String proxyUser = (String) configuration.get(ClientConfigProperties.PROXY_USER.getKey()); + String proxyPassword = (String) configuration.get(ClientConfigProperties.PROXY_PASSWORD.getKey()); + if (proxyUser != null && proxyPassword != null) { + proxyAuthHeaderValue = "Basic " + Base64.getEncoder().encodeToString((proxyUser + ":" + proxyPassword).getBytes(StandardCharsets.UTF_8)); + } + + } else if (proxyType == ProxyType.SOCKS) { + soCfgBuilder.setSocksProxyAddress(new InetSocketAddress(proxyHost, proxyPort)); + } + + boolean disableCookies = !((Boolean)ClientConfigProperties.HTTP_SAVE_COOKIES.getOrDefault(configuration)); + if (disableCookies) { + clientBuilder.disableCookieManagement(); + } + SocketConfig socketConfig = soCfgBuilder.build(); + + // Connection manager + if (ClientConfigProperties.CONNECTION_POOL_ENABLED.getOrDefault(configuration)) { + clientBuilder.setConnectionManager(poolConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); + } else { + clientBuilder.setConnectionManager(basicConnectionManager(sslConnectionSocketFactory, socketConfig, configuration)); + } + Long keepAliveTimeout = ClientConfigProperties.HTTP_KEEP_ALIVE_TIMEOUT.getOrDefault(configuration); + if (keepAliveTimeout != null && keepAliveTimeout > 0) { + clientBuilder.setKeepAliveStrategy((response, context) -> TimeValue.ofMilliseconds(keepAliveTimeout)); + } + + clientBuilder.disableContentCompression(); // will handle ourselves + + return clientBuilder.build(); + } + + private static final String ERROR_CODE_PREFIX_PATTERN = "%d. DB::Exception:"; + + /** + * Reads status line and if error tries to parse response body to get server error message. + * + * @param httpResponse - HTTP response + * @return exception object with server code + */ + public Exception readError(HttpPost req, ClassicHttpResponse httpResponse) { + final Header serverQueryIdHeader = httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); + final Header clientQueryIdHeader = req.getFirstHeader(ClickHouseHttpProto.HEADER_QUERY_ID); + final Header queryHeader = Stream.of(serverQueryIdHeader, clientQueryIdHeader).filter(Objects::nonNull).findFirst().orElse(null); + final String queryId = queryHeader == null ? "" : queryHeader.getValue(); + int serverCode = getHeaderInt(httpResponse.getFirstHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE), 0); + try { + return serverCode > 0 ? readClickHouseError(httpResponse.getEntity(), serverCode, queryId, httpResponse.getCode()) : + readNotClickHouseError(httpResponse.getEntity(), queryId, httpResponse.getCode()); + } catch (Exception e) { + LOG.error("Failed to read error message", e); + String msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpResponse.getCode() + ")"; + return new ServerException(serverCode, msg + " (queryId= " + queryId + ")", httpResponse.getCode(), queryId); + } + } + + private ServerException readNotClickHouseError(HttpEntity httpEntity, String queryId, int httpCode) { + + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + + String msg = null; + InputStream body = null; + int offset = 0; + for (int i = 0; i < 2; i++) { + try { + if (body == null) { + body = httpEntity.getContent(); + } + int msgLen = body.read(buffer, offset, buffer.length - offset); + if (msgLen > 0) { + msg = new String(buffer, 0, msgLen, StandardCharsets.UTF_8).trim(); + if (msg.isEmpty()) { + msg = ""; + } + } + break; + } catch (ClientException e) { + // Invalid LZ4 Magic + if (body instanceof ClickHouseLZ4InputStream) { + ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; + body = stream.getInputStream(); + byte[] lzHeader = stream.getHeaderBuffer(); // Here is read part of original body + offset = Math.min(lzHeader.length, buffer.length); + System.arraycopy(lzHeader, 0, buffer, 0, offset); + continue; + } + throw e; + } catch (Exception e) { + LOG.warn("Failed to read error message (queryId = " + queryId + ")", e); + break; + } + } + + String errormsg = msg == null ? "unknown server error" : msg; + return new ServerException(ServerException.CODE_UNKNOWN, errormsg + " (transport error: " + httpCode +")", httpCode, queryId); + } + + private static ServerException readClickHouseError(HttpEntity httpEntity, int serverCode, String queryId, int httpCode) throws Exception { + InputStream body = httpEntity.getContent(); + byte[] buffer = new byte[ERROR_BODY_BUFFER_SIZE]; + byte[] lookUpStr = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode).getBytes(StandardCharsets.UTF_8); + StringBuilder msgBuilder = new StringBuilder(); + boolean found = false; + while (true) { + int rBytes = -1; + try { + rBytes = body.read(buffer); + } catch (ClientException e) { + // Invalid LZ4 Magic + if (body instanceof ClickHouseLZ4InputStream) { + ClickHouseLZ4InputStream stream = (ClickHouseLZ4InputStream) body; + body = stream.getInputStream(); + byte[] headerBuffer = stream.getHeaderBuffer(); + System.arraycopy(headerBuffer, 0, buffer, 0, headerBuffer.length); + rBytes = headerBuffer.length; + } + } + if (rBytes == -1) { + break; + } + + for (int i = 0; i < rBytes; i++) { + if (buffer[i] == lookUpStr[0]) { + found = true; + for (int j = 1; j < Math.min(rBytes - i, lookUpStr.length); j++) { + if (buffer[i + j] != lookUpStr[j]) { + found = false; + break; + } + } + if (found) { + msgBuilder.append(new String(buffer, i, rBytes - i, StandardCharsets.UTF_8)); + break; + } + } + } + + if (found) { + break; + } + } + + while (true) { + int rBytes = body.read(buffer); + if (rBytes == -1) { + break; + } + msgBuilder.append(new String(buffer, 0, rBytes, StandardCharsets.UTF_8)); + } + + String msg = msgBuilder.toString().replaceAll("\\s+", " ").replaceAll("\\\\n", " ") + .replaceAll("\\\\/", "/"); + if (msg.trim().isEmpty()) { + msg = String.format(ERROR_CODE_PREFIX_PATTERN, serverCode) + " (transport error: " + httpCode + ")"; + } + return new ServerException(serverCode, "Code: " + msg + " (queryId= " + queryId + ")", httpCode, queryId); + } + + private static final long POOL_VENT_TIMEOUT = 10000L; + private final AtomicLong timeToPoolVent = new AtomicLong(0); + + private void doPoolVent() { + if (poolControl != null && timeToPoolVent.get() < System.currentTimeMillis()) { + timeToPoolVent.set(System.currentTimeMillis() + POOL_VENT_TIMEOUT); + poolControl.closeExpired(); + } + } + + private HttpContext createRequestHttpContext(Map requestConfig) { + HttpClientContext context = HttpClientContext.create(); + Number responseTimeout = ClientConfigProperties.SOCKET_OPERATION_TIMEOUT.getOrDefault(requestConfig); + Number connectionReqTimeout = ClientConfigProperties.CONNECTION_REQUEST_TIMEOUT.getOrDefault(requestConfig); + RequestConfig reqHttpConf = RequestConfig.custom() + .setResponseTimeout(responseTimeout.longValue(), TimeUnit.MILLISECONDS) + .setConnectionRequestTimeout(connectionReqTimeout.longValue(), TimeUnit.MILLISECONDS) + .build(); + context.setRequestConfig(reqHttpConf); + + return context; + } + + private URI createRequestURI(Endpoint server, Map requestConfig, boolean addParameters) { + URI uri; + try { + URIBuilder uriBuilder = new URIBuilder(server.getURI()); + addRequestParams(requestConfig, uriBuilder::addParameter); + + if (addParameters) { + addStatementParams(requestConfig, uriBuilder::addParameter); + } + + uri = uriBuilder.optimize().build(); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + return uri; + } + + private HttpPost createPostRequest(URI uri, Map requestConfig) { + HttpPost req = new HttpPost(uri); +// req.setVersion(new ProtocolVersion("HTTP", 1, 0)); // to disable chunk transfer encoding + addHeaders(req, requestConfig); + return req; + } + + public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, + String body) throws Exception { + + final URI uri = createRequestURI(server, requestConfig, true); + final HttpPost req = createPostRequest(uri, requestConfig); + final String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; + + HttpEntity httpEntity = new ByteArrayEntity(body.getBytes(StandardCharsets.UTF_8.name()), CONTENT_TYPE, contentEncoding); + req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); + + return doPostRequest(requestConfig, req); + } + + public ClassicHttpResponse executeMultiPartRequest(Endpoint server, Map requestConfig, String sqlQuery) throws Exception { + + requestConfig.put(ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getKey(), false); + + final URI uri = createRequestURI(server, requestConfig, false); + final HttpPost req = createPostRequest(uri, requestConfig); + + MultipartEntityBuilder multipartEntityBuilder = MultipartEntityBuilder.create(); + addStatementParams(requestConfig, multipartEntityBuilder::addTextBody); + multipartEntityBuilder.addTextBody(ClickHouseHttpProto.QPARAM_QUERY_STMT, sqlQuery); + + + HttpEntity httpEntity = multipartEntityBuilder.build(); + req.setHeader(HttpHeaders.CONTENT_TYPE, httpEntity.getContentType()); // set proper content type with generated boundary value + req.setEntity(wrapRequestEntity(httpEntity, requestConfig)); + + return doPostRequest(requestConfig, req); + + + } + + public ClassicHttpResponse executeRequest(Endpoint server, Map requestConfig, + IOCallback writeCallback) throws Exception { + + final URI uri = createRequestURI(server, requestConfig, true); + final HttpPost req = createPostRequest(uri, requestConfig); + String contentEncoding = req.containsHeader(HttpHeaders.CONTENT_ENCODING) ? req.getHeader(HttpHeaders.CONTENT_ENCODING).getValue() : null; + req.setEntity(wrapRequestEntity( + new EntityTemplate(-1, CONTENT_TYPE, contentEncoding , writeCallback), + requestConfig)); + + return doPostRequest(requestConfig, req); + } + + private ClassicHttpResponse doPostRequest(Map requestConfig, HttpPost req) throws Exception { + + doPoolVent(); + + ClassicHttpResponse httpResponse = null; + HttpContext context = createRequestHttpContext(requestConfig); + try { + httpResponse = httpClient.executeOpen(null, req, context); + + httpResponse.setEntity(wrapResponseEntity(httpResponse.getEntity(), + httpResponse.getCode(), + requestConfig)); + + if (httpResponse.getCode() == HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED) { + throw new ClientMisconfigurationException("Proxy authentication required. Please check your proxy settings."); + } else if (httpResponse.getCode() == HttpStatus.SC_BAD_GATEWAY) { + httpResponse.close(); + throw new ClientException("Server returned '502 Bad gateway'. Check network and proxy settings."); + } else if (httpResponse.getCode() >= HttpStatus.SC_BAD_REQUEST || httpResponse.containsHeader(ClickHouseHttpProto.HEADER_EXCEPTION_CODE)) { + try { + throw readError(req, httpResponse); + } finally { + httpResponse.close(); + } + } + return httpResponse; + + } catch (UnknownHostException e) { + closeQuietly(httpResponse); + LOG.warn("Host '{}' unknown", req.getAuthority()); + throw e; + } catch (ConnectException | NoRouteToHostException e) { + closeQuietly(httpResponse); + LOG.warn("Failed to connect to '{}': {}", req.getAuthority(), e.getMessage()); + throw e; + } catch (Exception e) { + closeQuietly(httpResponse); + LOG.debug("Failed to execute request to '{}': {}", req.getAuthority(), e.getMessage(), e); + throw e; + } + } + + public static void closeQuietly(ClassicHttpResponse httpResponse) { + if (httpResponse != null) { + try { + httpResponse.close(); + } catch (IOException e) { + LOG.warn("Failed to close response"); + } + } + } + + private static final ContentType CONTENT_TYPE = ContentType.create(ContentType.TEXT_PLAIN.getMimeType(), "UTF-8"); + + private void addHeaders(HttpPost req, Map requestConfig) { + setHeader(req, HttpHeaders.CONTENT_TYPE, CONTENT_TYPE.getMimeType()); + if (requestConfig.containsKey(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())) { + setHeader( + req, + ClickHouseHttpProto.HEADER_FORMAT, + ((ClickHouseFormat) requestConfig.get(ClientConfigProperties.INPUT_OUTPUT_FORMAT.getKey())).name()); + } + if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { + setHeader( + req, + ClickHouseHttpProto.HEADER_QUERY_ID, + (String) requestConfig.get(ClientConfigProperties.QUERY_ID.getKey())); + } + setHeader( + req, + ClickHouseHttpProto.HEADER_DATABASE, + ClientConfigProperties.DATABASE.getOrDefault(requestConfig)); + + if (ClientConfigProperties.SSL_AUTH.getOrDefault(requestConfig).booleanValue()) { + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_USER, + ClientConfigProperties.USER.getOrDefault(requestConfig)); + setHeader( + req, + ClickHouseHttpProto.HEADER_SSL_CERT_AUTH, + "on"); + } else if (ClientConfigProperties.HTTP_USE_BASIC_AUTH.getOrDefault(requestConfig).booleanValue()) { + String user = ClientConfigProperties.USER.getOrDefault(requestConfig); + String password = ClientConfigProperties.PASSWORD.getOrDefault(requestConfig); + if (password == null) { + password = ""; + } + // Use as-is, no encoding allowed + req.addHeader( + HttpHeaders.AUTHORIZATION, + "Basic " + Base64.getEncoder().encodeToString( + (user + ":" + password).getBytes(StandardCharsets.UTF_8))); + } else { + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_USER, + ClientConfigProperties.USER.getOrDefault(requestConfig)); + setHeader( + req, + ClickHouseHttpProto.HEADER_DB_PASSWORD, + ClientConfigProperties.PASSWORD.getOrDefault(requestConfig)); + } + if (proxyAuthHeaderValue != null) { + req.addHeader( + HttpHeaders.PROXY_AUTHORIZATION, + proxyAuthHeaderValue); + } + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); + + if (useHttpCompression) { + if (serverCompression) { + setHeader(req, HttpHeaders.ACCEPT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); + } + + if (clientCompression && !appCompressedData) { + setHeader(req, HttpHeaders.CONTENT_ENCODING, DEFAULT_HTTP_COMPRESSION_ALGO); + } + } + + for (String key : requestConfig.keySet()) { + if (key.startsWith(ClientConfigProperties.HTTP_HEADER_PREFIX)) { + Object val = requestConfig.get(key); + if (val != null) { + setHeader( + req, + key.substring(ClientConfigProperties.HTTP_HEADER_PREFIX.length()), + String.valueOf(val)); + } + } + } + + // Special cases + if (req.containsHeader(HttpHeaders.AUTHORIZATION) + && (req.containsHeader(ClickHouseHttpProto.HEADER_DB_USER) || + req.containsHeader(ClickHouseHttpProto.HEADER_DB_PASSWORD))) + { + // user has set auth header for purpose, lets remove ours + req.removeHeaders(ClickHouseHttpProto.HEADER_DB_USER); + req.removeHeaders(ClickHouseHttpProto.HEADER_DB_PASSWORD); + } + + // -- keep last + correctUserAgentHeader(req, requestConfig); + } + + private void addRequestParams(Map requestConfig, BiConsumer consumer) { + if (requestConfig.containsKey(ClientConfigProperties.QUERY_ID.getKey())) { + consumer.accept(ClickHouseHttpProto.QPARAM_QUERY_ID, requestConfig.get(ClientConfigProperties.QUERY_ID.getKey()).toString()); + } + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + + if (useHttpCompression) { + // enable_http_compression make server react on http header + // for client side compression Content-Encoding should be set + // for server side compression Accept-Encoding should be set + consumer.accept("enable_http_compression", "1"); + } else { + if (serverCompression) { + consumer.accept("compress", "1"); + } + if (clientCompression) { + consumer.accept("decompress", "1"); + } + } + + Collection sessionRoles = ClientConfigProperties.SESSION_DB_ROLES.getOrDefault(requestConfig); + if (!(sessionRoles == null || sessionRoles.isEmpty())) { + sessionRoles.forEach(r -> consumer.accept(ClickHouseHttpProto.QPARAM_ROLE, r)); + } + + for (String key : requestConfig.keySet()) { + if (key.startsWith(ClientConfigProperties.SERVER_SETTING_PREFIX)) { + Object val = requestConfig.get(key); + if (val != null) { + consumer.accept(key.substring(ClientConfigProperties.SERVER_SETTING_PREFIX.length()), String.valueOf(requestConfig.get(key))); + } + } + } + } + + private void addStatementParams(Map requestConfig, BiConsumer consumer) { + if (requestConfig.containsKey(KEY_STATEMENT_PARAMS)) { + Map params = (Map) requestConfig.get(KEY_STATEMENT_PARAMS); + params.forEach((k, v) -> consumer.accept("param_" + k, String.valueOf(v))); + } + } + + private HttpEntity wrapRequestEntity(HttpEntity httpEntity, Map requestConfig) { + + boolean clientCompression = ClientConfigProperties.COMPRESS_CLIENT_REQUEST.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + boolean appCompressedData = ClientConfigProperties.APP_COMPRESSED_DATA.getOrDefault(requestConfig); + + LOG.debug("wrapRequestEntity: client compression: {}, http compression: {}, content encoding: {}", + clientCompression, useHttpCompression, httpEntity.getContentEncoding()); + + if (httpEntity.getContentEncoding() != null && !appCompressedData) { + // http header is set and data is not compressed + return new CompressedEntity(httpEntity, false, CompressorStreamFactory.getSingleton()); + } else if (clientCompression && !appCompressedData) { + int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); + return new LZ4Entity(httpEntity, useHttpCompression, false, true, + buffSize, false, lz4Factory); + } else { + return httpEntity; + } + } + + private HttpEntity wrapResponseEntity(HttpEntity httpEntity, int httpStatus, Map requestConfig) { + boolean serverCompression = ClientConfigProperties.COMPRESS_SERVER_RESPONSE.getOrDefault(requestConfig); + boolean useHttpCompression = ClientConfigProperties.USE_HTTP_COMPRESSION.getOrDefault(requestConfig); + + LOG.debug("wrapResponseEntity: server compression: {}, http compression: {}, content encoding: {}", + serverCompression, useHttpCompression, httpEntity.getContentEncoding()); + + if (httpEntity.getContentEncoding() != null) { + // http compressed response + return new CompressedEntity(httpEntity, true, CompressorStreamFactory.getSingleton()); + } + + // data compression + if (serverCompression && !(httpStatus == HttpStatus.SC_FORBIDDEN || httpStatus == HttpStatus.SC_UNAUTHORIZED)) { + int buffSize = ClientConfigProperties.COMPRESSION_LZ4_UNCOMPRESSED_BUF_SIZE.getOrDefault(requestConfig); + return new LZ4Entity(httpEntity, useHttpCompression, true, false, buffSize, true, lz4Factory); + } + + return httpEntity; + } + + public static int getHeaderInt(Header header, int defaultValue) { + return getHeaderVal(header, defaultValue, Integer::parseInt); + } + + private static final Set RESPONSE_HEADER_WHITELIST = new HashSet<>(Arrays.asList( + ClickHouseHttpProto.HEADER_QUERY_ID, + ClickHouseHttpProto.HEADER_SRV_SUMMARY, + ClickHouseHttpProto.HEADER_SRV_DISPLAY_NAME, + ClickHouseHttpProto.HEADER_DATABASE, + ClickHouseHttpProto.HEADER_DB_USER + )); + + /** + * Collects whitelisted response headers from an HTTP response into a map. + * + * @param response the HTTP response + * @return unmodifiable map of header name to header value for whitelisted headers present in the response + */ + public static Map collectResponseHeaders(ClassicHttpResponse response) { + Map headers = new HashMap<>(); + for (String name : RESPONSE_HEADER_WHITELIST) { + Header header = response.getFirstHeader(name); + if (header != null) { + headers.put(name, header.getValue()); + } + } + return Collections.unmodifiableMap(headers); + } + + public static String getHeaderVal(Header header, String defaultValue) { + return getHeaderVal(header, defaultValue, Function.identity()); + } + + public static T getHeaderVal(Header header, T defaultValue, Function converter) { + if (header == null) { + return defaultValue; + } + + return converter.apply(header.getValue()); + } + + public boolean shouldRetry(Throwable ex, Map requestSettings) { + List retryCauses = ClientConfigProperties.CLIENT_RETRY_ON_FAILURE.getOrDefault(requestSettings); + + if (retryCauses.contains(ClientFaultCause.None)) { + return false; + } + + if (ex instanceof NoHttpResponseException + || ex.getCause() instanceof NoHttpResponseException) { + return retryCauses.contains(ClientFaultCause.NoHttpResponse); + } + + if (ex instanceof ConnectException + || ex instanceof ConnectTimeoutException + || ex.getCause() instanceof ConnectException + || ex.getCause() instanceof ConnectTimeoutException) { + return retryCauses.contains(ClientFaultCause.ConnectTimeout); + } + + if (ex instanceof ConnectionRequestTimeoutException + || ex.getCause() instanceof ConnectionRequestTimeoutException) { + return retryCauses.contains(ClientFaultCause.ConnectionRequestTimeout); + } + + if (ex instanceof SocketTimeoutException + || ex.getCause() instanceof SocketTimeoutException) { + return retryCauses.contains(ClientFaultCause.SocketTimeout); + } + + // there are some db retryable error codes + if (ex instanceof ServerException || ex.getCause() instanceof ServerException) { + ServerException se = (ServerException) ex; + return se.isRetryable() && retryCauses.contains(ClientFaultCause.ServerRetryable); + } + + return false; + } + + // This method wraps some client specific exceptions into specific ClientException or just ClientException + // ClientException will be also wrapped + public RuntimeException wrapException(String message, Exception cause, String queryId) { + if (cause instanceof ClientException || cause instanceof ServerException) { + return (RuntimeException) cause; + } + + if (cause instanceof ConnectionRequestTimeoutException || + cause instanceof NoHttpResponseException || + cause instanceof ConnectTimeoutException || + cause instanceof ConnectException || + cause instanceof UnknownHostException || + cause instanceof NoRouteToHostException) { + ConnectionInitiationException ex = new ConnectionInitiationException(message, cause); + ex.setQueryId(queryId); + return ex; + } + + if (cause instanceof SocketTimeoutException || cause instanceof IOException) { + DataTransferException ex = new DataTransferException(message, cause); + ex.setQueryId(queryId); + return ex; + } + // if we can not identify the exception explicitly we catch as our base exception ClickHouseException + return new ClickHouseException(message, cause, queryId); + } + + private void correctUserAgentHeader(HttpRequest request, Map requestConfig) { + //TODO: implement cache for user-agent + Header userAgentHeader = request.getLastHeader(HttpHeaders.USER_AGENT); + request.removeHeaders(HttpHeaders.USER_AGENT); + + String clientName = ClientConfigProperties.CLIENT_NAME.getOrDefault(requestConfig); + String userAgentValue = defaultUserAgent; + if (userAgentHeader == null && clientName != null && !clientName.isEmpty()) { + userAgentValue = clientName + " " + defaultUserAgent; + } else if (userAgentHeader != null) { + userAgentValue = userAgentHeader.getValue() + " " + defaultUserAgent; + } + request.setHeader(HttpHeaders.USER_AGENT, userAgentValue); + } + + private String buildDefaultUserAgent() { + StringBuilder userAgent = new StringBuilder(); + userAgent.append(Client.CLIENT_USER_AGENT); + + String clientVersion = Client.clientVersion; + + userAgent.append(clientVersion); + + userAgent.append(" ("); + userAgent.append(System.getProperty("os.name")); + userAgent.append("; "); + userAgent.append("jvm:").append(System.getProperty("java.version")); + userAgent.append("; "); + + userAgent.setLength(userAgent.length() - 2); + userAgent.append(')'); + + try { + String httpClientVersion = this.httpClient.getClass().getPackage().getImplementationVersion(); + if (Objects.equals(this.httpClient.getClass().getPackage().getImplementationTitle(), this.getClass().getPackage().getImplementationTitle())) { + // shaded jar - all packages have same implementation title + httpClientVersion = "unknown"; + try (InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream("client-v2-version.properties")) { + Properties p = new Properties(); + p.load(in); + + String tmp = p.getProperty("apache.http.client.version"); + if (tmp != null && !tmp.isEmpty() && !tmp.equals("${apache.httpclient.version}")) { + httpClientVersion = tmp; + } + } catch (Exception e) { + // ignore + } + } + userAgent.append(" ") + .append("Apache-HttpClient") + .append('/') + .append(httpClientVersion); + } catch (Exception e) { + LOG.info("failed to construct http client version string"); + } + return userAgent.toString(); + } + + public void close() { + httpClient.close(CloseMode.IMMEDIATE); + } + + private static void setHeader(HttpRequest req, String headerName, + String value) + { + if (value == null) { + return; + } + + if (value.trim().isEmpty()) { + return; + } + if (PATTERN_HEADER_VALUE_ASCII.matcher(value).matches()) { + req.setHeader(headerName, value); + } else { + try { + req.setHeader( + headerName + "*", + "UTF-8''" + URLEncoder.encode(value, StandardCharsets.UTF_8.name())); + } catch (UnsupportedEncodingException e) { + throw new ClientException("Failed to convert string to UTF8" , e); + } + } + } + + /** + * This factory is used only when no ssl connections are required (no https endpoints). + * Internally http client would create factory and spend time if no supplied. + */ + private static class DummySSLConnectionSocketFactory implements LayeredConnectionSocketFactory { + @Override + public Socket createLayeredSocket(Socket socket, String target, int port, HttpContext context) throws IOException { + return null; + } + + @Override + public Socket createSocket(HttpContext context) throws IOException { + return null; + } + + @Override + public Socket connectSocket(TimeValue connectTimeout, Socket socket, HttpHost host, InetSocketAddress remoteAddress, InetSocketAddress localAddress, HttpContext context) throws IOException { + return null; + } + } + + public static class MeteredManagedHttpClientConnectionFactory extends ManagedHttpClientConnectionFactory { + public MeteredManagedHttpClientConnectionFactory(Http1Config http1Config, CharCodingConfig charCodingConfig, DefaultHttpResponseParserFactory defaultHttpResponseParserFactory) { + super(http1Config, charCodingConfig, defaultHttpResponseParserFactory); + } + + ConcurrentLinkedQueue times = new ConcurrentLinkedQueue<>(); + + + @Override + public ManagedHttpClientConnection createConnection(Socket socket) throws IOException { + long startT = System.currentTimeMillis(); + try { + return super.createConnection(socket); + } finally { + long endT = System.currentTimeMillis(); + times.add(endT - startT); + } + } + + public long getTime() { + int count = times.size(); + long runningAverage = 0; + for (int i = 0; i < count; i++) { + Long t = times.poll(); + if (t != null) { + runningAverage += t; + } + } + + return count > 0 ? runningAverage / count : 0; + } + } + + public static class CustomSSLConnectionFactory extends SSLConnectionSocketFactory { + + private final SNIHostName defaultSNI; + + public CustomSSLConnectionFactory(String defaultSNI, SSLContext sslContext, HostnameVerifier hostnameVerifier) { + super(sslContext, hostnameVerifier); + this.defaultSNI = defaultSNI == null || defaultSNI.trim().isEmpty() ? null : new SNIHostName(defaultSNI); + } + + @Override + protected void prepareSocket(SSLSocket socket, HttpContext context) throws IOException { + super.prepareSocket(socket, context); + + if (defaultSNI != null) { + SSLParameters sslParams = socket.getSSLParameters(); + sslParams.setServerNames(Collections.singletonList(defaultSNI)); + socket.setSSLParameters(sslParams); + } + } + } +} diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java index 1cd75c7e3..8d765a130 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java @@ -7,6 +7,7 @@ import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpEntity; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java index 59f53594c..ed571bb7b 100644 --- a/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java +++ b/client-v2/src/test/java/com/clickhouse/client/HttpTransportTests.java @@ -1,1736 +1,1730 @@ -package com.clickhouse.client; - -import com.clickhouse.client.api.Client; -import com.clickhouse.client.api.ClientConfigProperties; -import com.clickhouse.client.api.ClientException; -import com.clickhouse.client.api.ClientFaultCause; -import com.clickhouse.client.api.ConnectionInitiationException; -import com.clickhouse.client.api.ConnectionReuseStrategy; -import com.clickhouse.client.api.ServerException; -import com.clickhouse.client.api.command.CommandResponse; -import com.clickhouse.client.api.command.CommandSettings; -import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; -import com.clickhouse.client.api.enums.Protocol; -import com.clickhouse.client.api.enums.ProxyType; -import com.clickhouse.client.api.insert.InsertResponse; -import com.clickhouse.client.api.internal.DataTypeConverter; -import com.clickhouse.client.api.internal.ServerSettings; -import com.clickhouse.client.api.query.GenericRecord; -import com.clickhouse.client.api.query.QueryResponse; -import com.clickhouse.client.api.query.QuerySettings; -import com.clickhouse.client.config.ClickHouseClientOption; -import com.clickhouse.data.ClickHouseFormat; -import com.github.tomakehurst.wiremock.WireMockServer; -import com.github.tomakehurst.wiremock.client.WireMock; -import com.github.tomakehurst.wiremock.common.ConsoleNotifier; -import com.github.tomakehurst.wiremock.common.Slf4jNotifier; -import com.github.tomakehurst.wiremock.core.WireMockConfiguration; -import com.github.tomakehurst.wiremock.http.Fault; -import com.github.tomakehurst.wiremock.http.trafficlistener.WiremockNetworkTrafficListener; -import org.apache.hc.core5.http.ConnectionClosedException; -import org.apache.hc.core5.http.ConnectionRequestTimeoutException; -import org.apache.hc.core5.http.HttpHeaders; -import org.apache.hc.core5.http.HttpStatus; -import org.apache.hc.core5.net.URIBuilder; -import org.testcontainers.utility.ThrowingFunction; -import org.testng.Assert; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.net.InetAddress; -import java.net.Socket; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.Base64; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Pattern; -import java.util.zip.GZIPOutputStream; - -import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; -import static org.testng.Assert.fail; - -@Test(groups = {"integration"}) -public class HttpTransportTests extends BaseIntegrationTest { - - @Test(groups = {"integration"},dataProvider = "testConnectionTTLProvider") - @SuppressWarnings("java:S2925") - public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSockets) throws Exception { - if (isCloud()) { - return; // skip cloud tests because of wiremock proxy. TODO: fix it - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - int proxyPort = new Random().nextInt(1000) + 10000; - ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(proxyPort) - .networkTrafficListener(connectionCounter) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - URIBuilder targetURI = new URIBuilder(server.getBaseUri()) - .setPath(""); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().proxiedFrom(targetURI.build().toString())).build()); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(getPassword()) - .addProxy(ProxyType.HTTP, "localhost", proxyPort); - if (connectionTtl != null) { - clientBuilder.setConnectionTTL(connectionTtl, ChronoUnit.MILLIS); - } - if (keepAlive != null) { - clientBuilder.setKeepAliveTimeout(keepAlive, ChronoUnit.MILLIS); - } - - try (Client client = clientBuilder.build()) { - List resp = client.queryAll("select 1"); - Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); - - try { - Thread.sleep(1000L); - } catch (InterruptedException e) { - Assert.fail("Unexpected exception", e); - } - - resp = client.queryAll("select 1"); - Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - Assert.assertEquals(connectionCounter.opened.get(), openSockets); - proxy.stop(); - } - } - - @DataProvider(name = "testConnectionTTLProvider") - public static Object[][] testConnectionTTLProvider() { - return new Object[][] { - { 1000L, null, 2 }, - { 2000L, null, 1 }, - { null, 2000L, 1 }, - { null, 500L, 2 }, - { 1000L, 0L, 2 }, - { 1000L, 3000L, 2} - }; - } - - private static class ConnectionCounterListener implements WiremockNetworkTrafficListener { - - private AtomicInteger opened = new AtomicInteger(0); - private AtomicInteger closed = new AtomicInteger(0); - - @Override - public void opened(Socket socket) { - opened.incrementAndGet(); - } - - @Override - public void incoming(Socket socket, ByteBuffer bytes) { - // ignore - } - - @Override - public void outgoing(Socket socket, ByteBuffer bytes) { - // ignore - } - - @Override - public void closed(Socket socket) { - closed.incrementAndGet(); - } - } - - @Test(groups = {"integration"}) - public void testConnectionRequestTimeout() { - if (isCloud()) { - return; // mocked server - } - - int serverPort = new Random().nextInt(1000) + 10000; - ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(serverPort) - .networkTrafficListener(connectionCounter) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withFixedDelay(5000) - .withStatus(HttpStatus.SC_NOT_FOUND)).build()); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort) - .setUsername("default") - .setPassword(getPassword()) - .retryOnFailures(ClientFaultCause.None) - .setMaxConnections(1) - .setOption(ClickHouseClientOption.ASYNC.getKey(), "true") - .setSocketTimeout(10000, ChronoUnit.MILLIS) - .setConnectionRequestTimeout(5, ChronoUnit.MILLIS); - - try (Client client = clientBuilder.build()) { - CompletableFuture f1 = client.query("select 1"); - Thread.sleep(500L); - CompletableFuture f2 = client.query("select 1"); - f2.get(); - } catch (ExecutionException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCause().getClass(), ConnectionInitiationException.class); - Assert.assertEquals(e.getCause().getCause().getClass(), ConnectionRequestTimeoutException.class); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - proxy.stop(); - } - } - - @Test(groups = {"integration"}) - public void testConnectionReuseStrategy() { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(getPassword()) - .setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO) - .build()) { - - List records = client.queryAll("SELECT timezone()"); - Assert.assertTrue(records.size() > 0); - Assert.assertEquals(records.get(0).getString(1), "UTC"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test(groups = { "integration" }) - public void testSecureConnection() { - if (isCloud()) { - return; // will fail in other tests - } - - ClickHouseNode secureServer = getSecureServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder() - .addEndpoint("https://localhost:" + secureServer.getPort()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .compressClientRequest(true) - .build()) { - - List records = client.queryAll("SELECT timezone()"); - Assert.assertTrue(records.size() > 0); - Assert.assertEquals(records.get(0).getString(1), "UTC"); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage()); - } - } - - @Test(groups = { "integration" }, dataProvider = "NoResponseFailureProvider") - public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, ThrowingFunction function, - boolean shouldFail) { - if (isCloud()) { - return; // mocked server - } - - WireMockServer faultyServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - faultyServer.start(); - - // First request gets no response - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withRequestBody(WireMock.equalTo(body)) - .inScenario("Retry") - .whenScenarioStateIs(STARTED) - .willSetStateTo("Failed") - .willReturn(WireMock.aResponse().withFault(Fault.EMPTY_RESPONSE)).build()); - - // Second request gets a response (retry) - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withRequestBody(WireMock.equalTo(body)) - .inScenario("Retry") - .whenScenarioStateIs("Failed") - .willSetStateTo("Done") - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - Client mockServerClient = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setMaxRetries(maxRetries) - .build(); - - try { - function.apply(mockServerClient); - } catch (ConnectionInitiationException e) { - e.printStackTrace(); - if (!shouldFail) { - Assert.fail("Unexpected exception", e); - } - return; - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } finally { - faultyServer.stop(); - } - - if (shouldFail) { - Assert.fail("Expected exception"); - } - } - - @DataProvider(name = "NoResponseFailureProvider") - public static Object[][] noResponseFailureProvider() { - - String insertBody = "1\t2\t3\n"; - ThrowingFunction insertFunction = (client) -> { - InsertResponse insertResponse = client.insert("table01", - new ByteArrayInputStream("1\t2\t3\n".getBytes()), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS); - insertResponse.close(); - return null; - }; - - String selectBody = "select timezone()"; - ThrowingFunction queryFunction = (client) -> { - QueryResponse response = client.query("select timezone()").get(30, TimeUnit.SECONDS); - response.close(); - return null; - }; - - return new Object[][]{ - {insertBody, 1, insertFunction, false}, - {selectBody, 1, queryFunction, false}, - {insertBody, 0, insertFunction, true}, - {selectBody, 0, queryFunction, true} - }; - } - - @Test(groups = { "integration" }, dataProvider = "testServerErrorHandlingDataProvider") - public void testServerErrorHandling(ClickHouseFormat format, boolean serverCompression, boolean useHttpCompression) { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(serverCompression) - .useHttpCompression(useHttpCompression) - .build()) { - - QuerySettings querySettings = new QuerySettings().setFormat(format); - try (QueryResponse response = - client.query("SELECT invalid;statement", querySettings).get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 62); - Assert.assertTrue(e.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"), - "Unexpected error message: " + e.getMessage()); - } - - - try (QueryResponse response = client.query("CREATE TABLE table_from_csv ENGINE MergeTree ORDER BY () AS SELECT * FROM file('empty.csv') ", querySettings) - .get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 636); - Assert.assertTrue(e.getMessage().contains("You can specify the structure manually: (in file/uri /var/lib/clickhouse/user_files/empty.csv). (CANNOT_EXTRACT_TABLE_STRUCTURE)"), - "Unexpected error message: " + e.getMessage()); - } - - querySettings.serverSetting("unknown_setting", "1"); - try (QueryResponse response = client.query("CREATE TABLE table_from_csv AS SELECT * FROM file('empty.csv')", querySettings) - .get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 115); - Assert.assertTrue(e.getMessage().startsWith("Code: 115. DB::Exception: Setting unknown_setting is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings. (UNKNOWN_SETTING)"), - "Unexpected error message: " + e.getMessage()); - } - - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(e.getMessage(), e); - } - - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("non-existing-user") - .setPassword("nothing") - .compressServerResponse(serverCompression) - .useHttpCompression(useHttpCompression) - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), 516); - Assert.assertTrue(e.getMessage().startsWith("Code: 516. DB::Exception: non-existing-user: Authentication failed: password is incorrect, or there is no user with such name. (AUTHENTICATION_FAILED)"), - e.getMessage()); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } - } - - @DataProvider(name = "testServerErrorHandlingDataProvider") - public static Object[][] testServerErrorHandlingDataProvider() { - EnumSet formats = EnumSet.of(ClickHouseFormat.CSV, ClickHouseFormat.TSV, - ClickHouseFormat.JSON, ClickHouseFormat.JSONCompact); - - int permutations = 3; - Object[][] result = new Object[formats.size() * permutations][]; - - int i = 0; - for (ClickHouseFormat format : formats) { - // format, server compression, http compression - result[i++] = new Object[]{format, false, false}; - result[i++] = new Object[]{format, true, false}; - result[i++] = new Object[]{format, true, true}; - } - - return result; - } - - @Test(groups = { "integration" }) - public void testErrorWithSuccessfulResponse() { - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withChunkedDribbleDelay(2, 200) - .withHeader("X-ClickHouse-Exception-Code", "241") - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")) - .build()); - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertTrue(e.getMessage().startsWith("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }, dataProvider = "testServerErrorsUncompressedDataProvider") - public void testServerErrorsUncompressed(int code, String message, String expectedMessage) { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withChunkedDribbleDelay(2, 200) - .withHeader("X-ClickHouse-Exception-Code", String.valueOf(code)) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(message)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.fail("Expected exception"); - } catch (ServerException e) { - e.printStackTrace(); - Assert.assertEquals(e.getCode(), code); - Assert.assertTrue(e.getMessage().startsWith(expectedMessage), "but started with " + e.getMessage()); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testServerErrorsUncompressedDataProvider") - public static Object[][] testServerErrorsUncompressedDataProvider() { - return new Object[][] { - { 241, "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB", - "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"}, - {900, "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full", - "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full"}, - {901, "Code: 901. DB::Exception: I write, erase, rewrite\n" + - "Erase again, and then\n" + - "A poppy blooms\n" + - " (by Katsushika Hokusai)", - "Code: 901. DB::Exception: I write, erase, rewrite " + - "Erase again, and then " + - "A poppy blooms" + - " (by Katsushika Hokusai)"} - }; - } - - @Test(groups = { "integration" }) - public void testAdditionalHeaders() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .httpHeader("X-ClickHouse-Test", "default_value") - .httpHeader("X-ClickHouse-Test-2", Arrays.asList("default_value1", "default_value2")) - .httpHeader("X-ClickHouse-Test-3", Arrays.asList("default_value1", "default_value2")) - .httpHeader("X-ClickHouse-Test-4", "default_value4") - .build()) { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("X-ClickHouse-Test", WireMock.equalTo("test")) - .withHeader("X-ClickHouse-Test-2", WireMock.equalTo("test1,test2")) - .withHeader("X-ClickHouse-Test-3", WireMock.equalTo("default_value1,default_value2")) - .withHeader("X-ClickHouse-Test-4", WireMock.equalTo("default_value4")) - - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - QuerySettings querySettings = new QuerySettings() - .httpHeader("X-ClickHouse-Test", "test") - .httpHeader("X-ClickHouse-Test-2", Arrays.asList("test1", "test2")); - - try (QueryResponse response = client.query("SELECT 1", querySettings).get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testServerSettings() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .serverSetting("max_threads", "10") - .serverSetting("async_insert", "1") - .serverSetting("roles", Arrays.asList("role1", "role2")) - .compressClientRequest(true) - .build()) { - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withQueryParam("max_threads", WireMock.equalTo("10")) - .withQueryParam("async_insert", WireMock.equalTo("3")) - .withQueryParam("roles", WireMock.equalTo("role3,role2")) - .withQueryParam("compress", WireMock.equalTo("0")) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - QuerySettings querySettings = new QuerySettings() - .serverSetting("max_threads", "10") - .serverSetting("async_insert", "3") - .serverSetting("roles", Arrays.asList("role3", "role2")) - .serverSetting("compress", "0"); - try (QueryResponse response = client.query("SELECT 1", querySettings).get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail("Unexpected exception", e); - } finally { - mockServer.stop(); - } - } - } - - static { - if (Boolean.getBoolean("test.debug")) { - System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); - } - } - - @Test(groups = { "integration" }) - public void testSSLAuthentication() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .setUsername("dba") - .setPassword("dba") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH ssl_certificate CN 'some_user'").get()) { - } - } - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .useSSLAuthentication(true) - .setUsername("some_user") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .setClientCertificate("some_user.crt") - .setClientKey("some_user.key") - .compressServerResponse(false) - .build()) { - - try (QueryResponse resp = client.query("SELECT 1").get()) { - Assert.assertEquals(resp.getReadRows(), 1); - } - } - } - - @Test(groups = { "integration" }, dataProvider = "testPasswordAuthenticationProvider", dataProviderClass = HttpTransportTests.class) - public void testPasswordAuthentication(String identifyWith, String identifyBy, boolean failsWithHeaders) throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("dba") - .setPassword("dba") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { - } - } catch (Exception e) { - Assert.fail("Failed on setup", e); - } - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .build()) { - - Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); - } catch (Exception e) { - Assert.fail("Failed to authenticate", e); - } - - if (failsWithHeaders) { - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .useHTTPBasicAuth(false) - .build()) { - - Assert.expectThrows(ClientException.class, () -> - client.queryAll("SELECT user()").get(0).getString(1)); - - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } - } - } - - @DataProvider(name = "testPasswordAuthenticationProvider") - public static Object[][] testPasswordAuthenticationProvider() { - return new Object[][] { - { "plaintext_password", "password", false}, - { "plaintext_password", "", false }, - { "plaintext_password", "S3Cr=?t", true}, - { "plaintext_password", "123§", true }, - { "sha256_password", "password", false }, - { "sha256_password", "123§", true }, - { "sha256_password", "S3Cr=?t", true}, - { "sha256_password", "S3Cr?=t", false}, - }; - } - - @Test(groups = { "integration" }) - public void testAuthHeaderIsKeptFromUser() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - - String identifyWith = "sha256_password"; - String identifyBy = "123§"; - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("dba") - .setPassword("dba") - .build()) { - - try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { - } - try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { - } - } catch (Exception e) { - Assert.fail("Failed on setup", e); - } - - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("some_user") - .setPassword(identifyBy) - .useHTTPBasicAuth(false) // disable basic auth to produce CH headers - .httpHeader(HttpHeaders.AUTHORIZATION, "Basic " + Base64.getEncoder().encodeToString(("some_user:" +identifyBy).getBytes())) - .build()) { - - Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); - } catch (Exception e) { - Assert.fail("Failed to authenticate", e); - } - } - - @Test(groups = { "integration" }) - public void testSSLAuthentication_invalidConfig() throws Exception { - if (isCloud()) { - return; // Current test is working only with local server because of self-signed certificates. - } - ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) - .useSSLAuthentication(true) - .setUsername("some_user") - .setPassword("s3cret") - .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") - .setClientCertificate("some_user.crt") - .setClientKey("some_user.key") - .compressServerResponse(false) - .build()) { - fail("Expected exception"); - } catch (IllegalArgumentException e) { - e.printStackTrace(); - Assert.assertTrue(e.getMessage().startsWith("Only one of password, access token or SSL authentication")); - } - } - - @Test(groups = { "integration" }) - public void testErrorWithSendProgressHeaders() throws Exception { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .build()) { - - try (CommandResponse resp = client.execute("DROP TABLE IF EXISTS test_omm_table").get()) { - } - try (CommandResponse resp = client.execute("CREATE TABLE test_omm_table ( val String) Engine = MergeTree ORDER BY () ").get()) { - } - - QuerySettings settings = new QuerySettings() - .serverSetting("send_progress_in_http_headers", "1") - .serverSetting("max_memory_usage", "54M"); - - try (QueryResponse resp = client.query("INSERT INTO test_omm_table SELECT randomString(16) FROM numbers(300000000)", settings).get()) { - - } catch (ServerException e) { - // 241 - MEMORY_LIMIT_EXCEEDED or 243 -NOT_ENOUGH_SPACE - Assert.assertTrue(e.getCode() == 241 || e.getCode() == 243); - } - } - } - - - @Test(groups = { "integration" }, dataProvider = "testUserAgentHasCompleteProductName_dataProvider", dataProviderClass = HttpTransportTests.class) - public void testUserAgentHasCompleteProductName(String clientName, Pattern userAgentPattern) throws Exception { - if (isCloud()) { - return; // mocked server - } - - ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); - try (Client client = new Client.Builder() - .addEndpoint(server.getBaseUri()) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setClientName(clientName) - .build()) { - - String q1Id = UUID.randomUUID().toString(); - - client.execute("SELECT 1", (CommandSettings) new CommandSettings().setQueryId(q1Id)).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT http_user_agent, http_referer, " + - " forwarded_for FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + q1Id + "'"); - Assert.assertFalse(logRecords.isEmpty(), "No records found in query log"); - - for (GenericRecord record : logRecords) { - System.out.println(record.getString("http_user_agent")); - Assert.assertTrue(userAgentPattern.matcher(record.getString("http_user_agent")).matches(), - record.getString("http_user_agent") + " doesn't match \"" + - userAgentPattern.pattern() + "\""); - - } - } - } - - - @DataProvider(name = "testUserAgentHasCompleteProductName_dataProvider") - public static Object[][] testUserAgentHasCompleteProductName_dataProvider() { - return new Object[][] { - { "", Pattern.compile("clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$") }, - { "test-client/1.0", Pattern.compile("test-client/1.0 clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}, - { "test-client/", Pattern.compile("test-client/ clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}}; - } - - @Test(dataProvider = "testClientNameDataProvider") - public void testClientName(String clientName, boolean setWithUpdate, String userAgentHeader, boolean setForRequest) throws Exception { - - final String initialClientName = setWithUpdate ? "init clientName" : clientName; - final String initialUserAgentHeader = setForRequest ? "init userAgentHeader" : userAgentHeader; - final String clientReferer = "http://localhost/webpage"; - - Client.Builder builder = newClient(); - if (initialClientName != null) { - builder.setClientName(initialClientName); - } - if (initialUserAgentHeader != null) { - builder.httpHeader(HttpHeaders.USER_AGENT, initialUserAgentHeader); - } - try (Client client = builder.build()) { - String expectedClientNameStartsWith = initialClientName == null || initialUserAgentHeader != null ? initialUserAgentHeader : initialClientName; - - if (setWithUpdate) { - client.updateClientName(clientName); - expectedClientNameStartsWith = initialUserAgentHeader == null ? clientName : initialUserAgentHeader; - } - - String qId = UUID.randomUUID().toString(); - QuerySettings settings = new QuerySettings() - .httpHeader(HttpHeaders.REFERER, clientReferer) - .setQueryId(qId); - - if (setForRequest) { - settings.httpHeader(HttpHeaders.USER_AGENT, userAgentHeader); - expectedClientNameStartsWith = userAgentHeader; - } - - client.query("SELECT 1", settings).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + - " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); - Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); - final String logUserAgent = logRecords.get(0).getString("http_user_agent"); - Assert.assertTrue(logUserAgent.startsWith(expectedClientNameStartsWith), - "Expected to start with \"" + expectedClientNameStartsWith + "\" but values was \"" + logUserAgent + "\"" ); - Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); - Assert.assertEquals(logRecords.get(0).getString("http_referer"), clientReferer); - Assert.assertEquals(logRecords.get(0).getString("client_name"), ""); // http client can't set this field - } - } - - @DataProvider(name = "testClientNameDataProvider") - public static Object[][] testClientName() { - return new Object[][] { - {"test-product (app 1.0)", false, null, false}, // only client name set - {"test-product (app 1.0)", false, "final product (app 1.1)", false}, // http header set and overrides client name - {"test-product (app 1.0)", true, null, false}, // client name set thru Client#updateClientName - {"test-product (app 1.0)", true, "final product (app 1.1)", true}, // custom UserAgent header overrides client name - }; - } - - @Test(dataProvider = "testClientNameThruRawOptionsDataProvider") - public void testClientNameThruRawOptions(String property, String value, boolean setInClient) throws Exception { - Client.Builder builder = newClient(); - if (setInClient) { - builder.setOption(property, value); - } - try (Client client = builder.build()) { - - String qId = UUID.randomUUID().toString(); - QuerySettings settings = new QuerySettings() - .setQueryId(qId); - - if (!setInClient) { - settings.setOption(property, value); - } - - client.query("SELECT 1", settings).get().close(); - client.execute("SYSTEM FLUSH LOGS").get().close(); - - List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + - " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); - Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); - final String logUserAgent = logRecords.get(0).getString("http_user_agent"); - Assert.assertTrue(logUserAgent.startsWith(value), - "Expected to start with \"" + value + "\" but values was \"" + logUserAgent + "\"" ); - Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); - } - } - - @DataProvider(name = "testClientNameThruRawOptionsDataProvider") - public Object[][] testClientNameThruRawOptionsDataProvider() { - return new Object[][] { - {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", true}, - {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", true}, - {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", false}, - {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", false}, - }; - } - - @Test(groups = { "integration" }) - public void testBearerTokenAuth() throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - String jwtToken1 = Arrays.stream( - new String[]{"header", "payload", "signature"}) - .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) - .reduce((s1, s2) -> s1 + "." + s2).get(); - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(jwtToken1) - .compressServerResponse(false) - .build()) { - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Unexpected exception", e); - } - } - - String jwtToken2 = Arrays.stream( - new String[]{"header2", "payload2", "signature2"}) - .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) - .reduce((s1, s2) -> s1 + "." + s2).get(); - - mockServer.resetAll(); - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_UNAUTHORIZED)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .useBearerTokenAuth(jwtToken1) - .compressServerResponse(false) - .build()) { - - try { - client.execute("SELECT 1").get(); - fail("Exception expected"); - } catch (ServerException e) { - Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); - } - - mockServer.resetAll(); - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) - - .build()); - - client.updateBearerToken(jwtToken2); - - client.execute("SELECT 1").get(); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testBasicAuthWithNoPassword() throws Exception { - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - // Expected: "default:" with empty password, not "default:null" - String expectedAuth = "Basic " + Base64.getEncoder() - .encodeToString("default:".getBytes(StandardCharsets.UTF_8)); - - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .withHeader("Authorization", WireMock.equalTo(expectedAuth)) - .willReturn(WireMock.aResponse() - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .compressServerResponse(false) - // no setPassword() call — password should default to empty, not "null" - .build()) { - - try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 10); - } catch (Exception e) { - Assert.fail("Basic auth with no password should send empty password, not 'null'", e); - } - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = { "integration" }) - public void testJWTWithCloud() throws Exception { - if (!isCloud()) { - return; // only for cloud - } - String jwt = System.getenv("CLIENT_JWT"); - Assert.assertTrue(jwt != null && !jwt.trim().isEmpty(), "JWT is missing"); - Assert.assertFalse(jwt.contains("\n") || jwt.contains("-----"), "JWT should be single string ready for HTTP header"); - try (Client client = newClient().useBearerTokenAuth(jwt).build()) { - try { - List response = client.queryAll("SELECT user(), now()"); - System.out.println("response: " + response.get(0).getString(1) + " time: " + response.get(0).getString(2)); - } catch (Exception e) { - e.printStackTrace(); - throw e; - } - } - } - - @Test(groups = { "integration" }) - public void testWithDefaultTimeouts() { - if (isCloud()) { - return; // mocked server - } - - int proxyPort = new Random().nextInt(1000) + 10000; - WireMockServer proxy = new WireMockServer(WireMockConfiguration - .options().port(proxyPort) - .notifier(new Slf4jNotifier(true))); - proxy.start(); - proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withFixedDelay(5000) - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", proxyPort, false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .build()) { - int startTime = (int) System.currentTimeMillis(); - try { - client.query("SELECT 1").get(); - } catch (Exception e) { - Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); - } - } finally { - proxy.stop(); - } - } - - - @Test(groups = { "integration" }) - public void testTimeoutsWithRetry() { - if (isCloud()) { - return; // mocked server - } - - WireMockServer faultyServer = new WireMockServer( WireMockConfiguration - .options().port(9090).notifier(new ConsoleNotifier(false))); - faultyServer.start(); - - // First request gets no response - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .inScenario("Timeout") - .withRequestBody(WireMock.containing("SELECT 1")) - .whenScenarioStateIs(STARTED) - .willSetStateTo("Failed") - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withFixedDelay(5000) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - // Second request gets a response (retry) - faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .inScenario("Timeout") - .withRequestBody(WireMock.containing("SELECT 1")) - .whenScenarioStateIs("Failed") - .willSetStateTo("Done") - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withFixedDelay(1000) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setSocketTimeout(3000) - .retryOnFailures(ClientFaultCause.SocketTimeout) - .build()) { - int startTime = (int) System.currentTimeMillis(); - try { - client.query("SELECT 1").get(); - } catch (Exception e) { - Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); - } - } finally { - faultyServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testSNIWithCloud() throws Exception { - if (!isCloud()) { - // skip for local env - return; - } - - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - String ip = InetAddress.getByName(node.getHost()).getHostAddress(); - try (Client c = new Client.Builder() - .addEndpoint(Protocol.HTTP, ip, node.getPort(), true) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .sslSocketSNI(node.getHost()).build()) { - c.execute("SELECT 1"); - } - } - - @Test(groups = {"integration"}) - public void testEndpointUrlPathIsPreserved() throws Exception { - if (isCloud()) { - return; // mocked server - } - - int serverPort = new Random().nextInt(1000) + 10000; - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().port(serverPort) - .notifier(new Slf4jNotifier(true))); - mockServer.start(); - - try { - // Setup stubs for two virtual ClickHouse instances behind a reverse proxy - mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales/db")) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); - - mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing/db")) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", - "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); - - // Test sales virtual instance - try (Client salesClient = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort + "/sales/db") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = salesClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 100); - } - } - - // Test billing virtual instance - also verify query parameters in URL are ignored - try (Client billingClient = new Client.Builder() - .addEndpoint("http://localhost:" + serverPort + "/billing/db?ignored_param=value") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - try (QueryResponse response = billingClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { - Assert.assertEquals(response.getReadBytes(), 200); - } - - // Verify that ignored_param is not in the request URL - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db")) - .withoutQueryParam("ignored_param")); - } - - // Verify requests were made to the correct paths - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales/db"))); - mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db"))); - - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testMultiPartRequest() { - final Map params = new HashMap<>(); - params.put("database_name", "system"); - params.put("table_names", - DataTypeConverter.INSTANCE.arrayToString(Arrays.asList("COLLATIONS", "ENGINES"), "Array(String)")); - - // Use http compression - try (Client client = newClient().useHttpCompression(true).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - - // Use http compression - try (Client client = newClient().useHttpCompression(false).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - - // compress request - try (Client client = newClient() - .compressClientRequest(true) - .setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true") - .useHttpCompression(true).build()) { - List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", - params); - - Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); - Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); - } - } - - @Test(groups = {"integration"}) - public void testNotFoundError() { - if (isCloud()) { - return; // not needed - } - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - - Client.Builder clientBuilder = new Client.Builder() - .addEndpoint("http://" + node.getHost() + ":" + node.getPort() + "/some-path") - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); - - try (Client client = clientBuilder.build()) { - client.queryAll("select 1"); - fail("Exception expected"); - } catch (ClientException e) { - Assert.assertTrue(e.getCause().getMessage().startsWith("There is no handle /some-path?")); - } - - } - - @Test(groups = {"integration"}) - public void testSmallNetworkBufferDoesNotBreakColumnDecoding() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 2_000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - - Assert.assertTrue(validBody.length > 3, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - 5); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, - "Expected failure when reading truncated binary stream"); - assertBinaryReadFailureContainsColumnName(thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testChunkedResponsePrematureEndIsReported() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT 10"; - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader(HttpHeaders.TRANSFER_ENCODING, "chunked") - .withFault(Fault.MALFORMED_RESPONSE_CHUNK)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, null, 30, - "Expected failure when reading malformed chunked response"); - ConnectionClosedException connectionClosedException = findCause(thrown, ConnectionClosedException.class); - boolean hasChunkedPrematureCloseSignature = containsMessageInCauseChain(thrown, - "closing chunk expected", - "premature end of chunk coded message body", - "failed to read header"); - Assert.assertTrue(connectionClosedException != null || hasChunkedPrematureCloseSignature, - "Expected chunked/premature-close failure signature, but was: " + thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testTailCorruptedStreamFailsDecoding() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 100_000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - - final int removedBytes = 3; - Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, - "Expected failure when reading binary stream truncated at tail"); - assertBinaryReadFailureContainsColumnName(thrown); - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}) - public void testTailStreamFailureReportsPositiveTimeSinceLastNextCall() throws Exception { - if (isCloud()) { - return; // mocked server - } - - final int rowsToRead = 2000; - final int networkBufferSize = 8196; - final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; - byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); - final int removedBytes = 3; - Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); - byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(HttpStatus.SC_OK) - .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") - .withBody(corruptedBody)) - .build()); - - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newMockServerClient(mockServer.port(), networkBufferSize) - .build(); - QueryResponse response = client.query(query, querySettings).get(60, TimeUnit.SECONDS); - ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { - final int[] rowsRead = new int[] {0}; - Throwable thrown = Assert.expectThrows(Throwable.class, () -> { - while (true) { - if (rowsRead[0] >= 5) { - try { - Thread.sleep(25); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - if (reader.next() == null) { - return; - } - rowsRead[0]++; - } - }); - - Assert.assertTrue(rowsRead[0] >= 5, - "Expected to read at least a few rows before failure, but read " + rowsRead[0]); - ClientException clientException = findCause(thrown, ClientException.class); - Assert.assertNotNull(clientException, - "Expected ClientException in cause chain, but was: " + thrown); - Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), - "Expected column information in failure message chain, but was: " + thrown); - - String elapsedTimeMessage = findFirstMessageInCauseChain(thrown, "time since last next call"); - Assert.assertNotNull(elapsedTimeMessage, - "Expected elapsed-time fragment in failure message chain, but was: " + thrown); - - java.util.regex.Matcher matcher = Pattern.compile("time since last next call (\\d+)\\)") - .matcher(elapsedTimeMessage); - Assert.assertTrue(matcher.find(), - "Expected elapsed-time fragment in message: " + elapsedTimeMessage); - long elapsedSinceLastNext = Long.parseLong(matcher.group(1)); - Assert.assertTrue(elapsedSinceLastNext > 0, - "Expected positive elapsed time since last next call, but was " + elapsedSinceLastNext); - } - } finally { - mockServer.stop(); - } - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") - public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(httpStatus) - .withBody(responseBody)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), - "Expected to contain '" + expectedBodyPart + "', but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusErrorBodyDataProvider") - public static Object[][] testHttpStatusErrorBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: invalid credentials for user default", "invalid credentials"}, - {HttpStatus.SC_FORBIDDEN, "Forbidden: user default has no access to this operation", "no access"}, - {HttpStatus.SC_NOT_FOUND, "Not found: requested endpoint does not exist", "requested endpoint"} - }; - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusWithoutBodyDataProvider") - public void testHttpStatusErrorsWithoutBody(int httpStatus) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse().withStatus(httpStatus)) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressServerResponse(false) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains("unknown server error"), - "Expected unknown error message for empty body, but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusWithoutBodyDataProvider") - public static Object[][] testHttpStatusWithoutBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED}, - {HttpStatus.SC_FORBIDDEN}, - {HttpStatus.SC_NOT_FOUND} - }; - } - - @Test(groups = {"integration"}, dataProvider = "testHttpStatusCompressedBodyDataProvider") - public void testHttpStatusErrorsWithHttpCompression(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { - if (isCloud()) { - return; // mocked server - } - - WireMockServer mockServer = new WireMockServer(WireMockConfiguration - .options().dynamicPort().notifier(new ConsoleNotifier(false))); - mockServer.start(); - - try { - mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) - .willReturn(WireMock.aResponse() - .withStatus(httpStatus) - .withHeader(HttpHeaders.CONTENT_ENCODING, "gzip") - .withBody(gzip(responseBody))) - .build()); - - try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .useHttpCompression(true) - .compressServerResponse(true) - .build()) { - - Throwable thrown = Assert.expectThrows(Throwable.class, - () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); - ServerException serverException = findServerException(thrown); - Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); - Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); - Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), - "Expected compressed body part '" + expectedBodyPart + "', but was: " + serverException.getMessage()); - } - } finally { - mockServer.stop(); - } - } - - @DataProvider(name = "testHttpStatusCompressedBodyDataProvider") - public static Object[][] testHttpStatusCompressedBodyDataProvider() { - return new Object[][]{ - {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: token is expired", "token is expired"}, - {HttpStatus.SC_FORBIDDEN, "Forbidden: policy denies this query", "policy denies"}, - {HttpStatus.SC_NOT_FOUND, "Not found: route does not exist", "route does not exist"} - }; - } - - private byte[] fetchBinaryPayload(String query, int networkBufferSize, int timeoutSec) throws Exception { - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newClient() - .useHttpCompression(false) - .compressServerResponse(false) - .setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)) - .build(); - QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS)) { - return readAllBytes(response.getInputStream()); - } - } - - private Throwable assertBinaryDecodeFails(WireMockServer mockServer, String query, Integer networkBufferSize, - int timeoutSec, String assertionMessage) throws Exception { - QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); - try (Client client = newMockServerClient(mockServer.port(), networkBufferSize).build()) { - Throwable thrown = Assert.expectThrows(Throwable.class, () -> { - try (QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS); - ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { - readAllRows(reader); - } - }); - Assert.assertNotNull(thrown, assertionMessage); - return thrown; - } - } - - private Client.Builder newMockServerClient(int port, Integer networkBufferSize) { - Client.Builder builder = new Client.Builder() - .addEndpoint(Protocol.HTTP, "localhost", port, false) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1") - .useHttpCompression(false) - .compressServerResponse(false); - if (networkBufferSize != null) { - builder.setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)); - } - return builder; - } - - private static void readAllRows(ClickHouseBinaryFormatReader reader) { - while (reader.next() != null) { - reader.getInteger(1); - reader.getLong(2); - reader.getString(3); - } - } - - private static byte[] gzip(String value) throws Exception { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { - gzipOutputStream.write(value.getBytes(StandardCharsets.UTF_8)); - } - return out.toByteArray(); - } - - private static byte[] readAllBytes(java.io.InputStream inputStream) throws Exception { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - byte[] buffer = new byte[8192]; - int read; - while ((read = inputStream.read(buffer)) != -1) { - out.write(buffer, 0, read); - } - return out.toByteArray(); - } - - private static ServerException findServerException(Throwable throwable) { - Throwable current = throwable; - while (current != null) { - if (current instanceof ServerException) { - return (ServerException) current; - } - current = current.getCause(); - } - return null; - } - - private static T findCause(Throwable throwable, Class clazz) { - Throwable current = throwable; - while (current != null) { - if (clazz.isInstance(current)) { - return clazz.cast(current); - } - current = current.getCause(); - } - return null; - } - - private static boolean containsMessageInCauseChain(Throwable throwable, String... parts) { - Throwable current = throwable; - while (current != null) { - String message = current.getMessage(); - if (message != null) { - String lower = message.toLowerCase(); - for (String part : parts) { - if (lower.contains(part.toLowerCase())) { - return true; - } - } - } - current = current.getCause(); - } - return false; - } - - private static String findFirstMessageInCauseChain(Throwable throwable, String part) { - Throwable current = throwable; - while (current != null) { - String message = current.getMessage(); - if (message != null && message.contains(part)) { - return message; - } - current = current.getCause(); - } - return null; - } - - private static void assertBinaryReadFailureContainsColumnName(Throwable thrown) { - ClientException clientException = findCause(thrown, ClientException.class); - Assert.assertNotNull(clientException, - "Expected ClientException in cause chain, but was: " + thrown); - Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), - "Expected column information in failure message chain, but was: " + thrown); - } - - protected Client.Builder newClient() { - ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); - boolean isSecure = isCloud(); - return new Client.Builder() - .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) - .setUsername("default") - .setPassword(ClickHouseServerForTest.getPassword()) - .compressClientRequest(false) - .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) - .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); - } -} +package com.clickhouse.client; + +import com.clickhouse.client.api.Client; +import com.clickhouse.client.api.ClientConfigProperties; +import com.clickhouse.client.api.ClientException; +import com.clickhouse.client.api.ClientFaultCause; +import com.clickhouse.client.api.ConnectionInitiationException; +import com.clickhouse.client.api.ConnectionReuseStrategy; +import com.clickhouse.client.api.ServerException; +import com.clickhouse.client.api.command.CommandResponse; +import com.clickhouse.client.api.command.CommandSettings; +import com.clickhouse.client.api.data_formats.ClickHouseBinaryFormatReader; +import com.clickhouse.client.api.enums.Protocol; +import com.clickhouse.client.api.enums.ProxyType; +import com.clickhouse.client.api.insert.InsertResponse; +import com.clickhouse.client.api.internal.DataTypeConverter; +import com.clickhouse.client.api.internal.ServerSettings; +import com.clickhouse.client.api.query.GenericRecord; +import com.clickhouse.client.api.query.QueryResponse; +import com.clickhouse.client.api.query.QuerySettings; +import com.clickhouse.client.config.ClickHouseClientOption; +import com.clickhouse.data.ClickHouseFormat; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.client.WireMock; +import com.github.tomakehurst.wiremock.common.ConsoleNotifier; +import com.github.tomakehurst.wiremock.common.Slf4jNotifier; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.http.Fault; +import com.github.tomakehurst.wiremock.http.trafficlistener.WiremockNetworkTrafficListener; +import org.apache.hc.core5.http.ConnectionClosedException; +import org.apache.hc.core5.http.ConnectionRequestTimeoutException; +import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.net.URIBuilder; +import org.testcontainers.utility.ThrowingFunction; +import org.testng.Assert; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.net.InetAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.Base64; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; +import java.util.zip.GZIPOutputStream; + +import static com.github.tomakehurst.wiremock.stubbing.Scenario.STARTED; +import static org.testng.Assert.fail; + +@Test(groups = {"integration"}) +public class HttpTransportTests extends BaseIntegrationTest { + + @Test(groups = {"integration"},dataProvider = "testConnectionTTLProvider") + @SuppressWarnings("java:S2925") + public void testConnectionTTL(Long connectionTtl, Long keepAlive, int openSockets) throws Exception { + if (isCloud()) { + return; // skip cloud tests because of wiremock proxy. TODO: fix it + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + int proxyPort = new Random().nextInt(1000) + 10000; + ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(proxyPort) + .networkTrafficListener(connectionCounter) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + URIBuilder targetURI = new URIBuilder(server.getBaseUri()) + .setPath(""); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().proxiedFrom(targetURI.build().toString())).build()); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(getPassword()) + .addProxy(ProxyType.HTTP, "localhost", proxyPort); + if (connectionTtl != null) { + clientBuilder.setConnectionTTL(connectionTtl, ChronoUnit.MILLIS); + } + if (keepAlive != null) { + clientBuilder.setKeepAliveTimeout(keepAlive, ChronoUnit.MILLIS); + } + + try (Client client = clientBuilder.build()) { + List resp = client.queryAll("select 1"); + Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); + + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + Assert.fail("Unexpected exception", e); + } + + resp = client.queryAll("select 1"); + Assert.assertEquals(resp.stream().findFirst().get().getString(1), "1"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + Assert.assertEquals(connectionCounter.opened.get(), openSockets); + proxy.stop(); + } + } + + @DataProvider(name = "testConnectionTTLProvider") + public static Object[][] testConnectionTTLProvider() { + return new Object[][] { + { 1000L, null, 2 }, + { 2000L, null, 1 }, + { null, 2000L, 1 }, + { null, 500L, 2 }, + { 1000L, 0L, 2 }, + { 1000L, 3000L, 2} + }; + } + + private static class ConnectionCounterListener implements WiremockNetworkTrafficListener { + + private AtomicInteger opened = new AtomicInteger(0); + private AtomicInteger closed = new AtomicInteger(0); + + @Override + public void opened(Socket socket) { + opened.incrementAndGet(); + } + + @Override + public void incoming(Socket socket, ByteBuffer bytes) { + // ignore + } + + @Override + public void outgoing(Socket socket, ByteBuffer bytes) { + // ignore + } + + @Override + public void closed(Socket socket) { + closed.incrementAndGet(); + } + } + + @Test(groups = {"integration"}) + public void testConnectionRequestTimeout() { + if (isCloud()) { + return; // mocked server + } + + int serverPort = new Random().nextInt(1000) + 10000; + ConnectionCounterListener connectionCounter = new ConnectionCounterListener(); + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(serverPort) + .networkTrafficListener(connectionCounter) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withFixedDelay(5000) + .withStatus(HttpStatus.SC_NOT_FOUND)).build()); + + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort) + .setUsername("default") + .setPassword(getPassword()) + .retryOnFailures(ClientFaultCause.None) + .setMaxConnections(1) + .setOption(ClickHouseClientOption.ASYNC.getKey(), "true") + .setSocketTimeout(10000, ChronoUnit.MILLIS) + .setConnectionRequestTimeout(5, ChronoUnit.MILLIS); + + try (Client client = clientBuilder.build()) { + CompletableFuture f1 = client.query("select 1"); + Thread.sleep(500L); + CompletableFuture f2 = client.query("select 1"); + f2.get(); + } catch (ExecutionException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCause().getClass(), ConnectionInitiationException.class); + Assert.assertEquals(e.getCause().getCause().getClass(), ConnectionRequestTimeoutException.class); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + proxy.stop(); + } + } + + @Test(groups = {"integration"}) + public void testConnectionReuseStrategy() { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(getPassword()) + .setConnectionReuseStrategy(ConnectionReuseStrategy.LIFO) + .build()) { + + List records = client.queryAll("SELECT timezone()"); + Assert.assertTrue(records.size() > 0); + Assert.assertEquals(records.get(0).getString(1), "UTC"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } + + @Test(groups = { "integration" }) + public void testSecureConnection() { + if (isCloud()) { + return; // will fail in other tests + } + + ClickHouseNode secureServer = getSecureServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder() + .addEndpoint("https://localhost:" + secureServer.getPort()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .compressClientRequest(true) + .build()) { + + List records = client.queryAll("SELECT timezone()"); + Assert.assertTrue(records.size() > 0); + Assert.assertEquals(records.get(0).getString(1), "UTC"); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + } + + @Test(groups = { "integration" }, dataProvider = "NoResponseFailureProvider") + public void testInsertAndNoHttpResponseFailure(String body, int maxRetries, ThrowingFunction function, + boolean shouldFail) { + if (isCloud()) { + return; // mocked server + } + + WireMockServer faultyServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + faultyServer.start(); + + // First request gets no response + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withRequestBody(WireMock.equalTo(body)) + .inScenario("Retry") + .whenScenarioStateIs(STARTED) + .willSetStateTo("Failed") + .willReturn(WireMock.aResponse().withFault(Fault.EMPTY_RESPONSE)).build()); + + // Second request gets a response (retry) + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withRequestBody(WireMock.equalTo(body)) + .inScenario("Retry") + .whenScenarioStateIs("Failed") + .willSetStateTo("Done") + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + Client mockServerClient = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setMaxRetries(maxRetries) + .build(); + + try { + function.apply(mockServerClient); + } catch (ConnectionInitiationException e) { + e.printStackTrace(); + if (!shouldFail) { + Assert.fail("Unexpected exception", e); + } + return; + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } finally { + faultyServer.stop(); + } + + if (shouldFail) { + Assert.fail("Expected exception"); + } + } + + @DataProvider(name = "NoResponseFailureProvider") + public static Object[][] noResponseFailureProvider() { + + String insertBody = "1\t2\t3\n"; + ThrowingFunction insertFunction = (client) -> { + InsertResponse insertResponse = client.insert("table01", + new ByteArrayInputStream("1\t2\t3\n".getBytes()), ClickHouseFormat.TSV).get(30, TimeUnit.SECONDS); + insertResponse.close(); + return null; + }; + + String selectBody = "select timezone()"; + ThrowingFunction queryFunction = (client) -> { + QueryResponse response = client.query("select timezone()").get(30, TimeUnit.SECONDS); + response.close(); + return null; + }; + + return new Object[][]{ + {insertBody, 1, insertFunction, false}, + {selectBody, 1, queryFunction, false}, + {insertBody, 0, insertFunction, true}, + {selectBody, 0, queryFunction, true} + }; + } + + @Test(groups = { "integration" }, dataProvider = "testServerErrorHandlingDataProvider") + public void testServerErrorHandling(ClickHouseFormat format, boolean serverCompression, boolean useHttpCompression) { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(serverCompression) + .useHttpCompression(useHttpCompression) + .build()) { + + QuerySettings querySettings = new QuerySettings().setFormat(format); + try (QueryResponse response = + client.query("SELECT invalid;statement", querySettings).get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 62); + Assert.assertTrue(e.getMessage().startsWith("Code: 62. DB::Exception: Syntax error (Multi-statements are not allowed): failed at position 15 (end of query)"), + "Unexpected error message: " + e.getMessage()); + } + + + try (QueryResponse response = client.query("CREATE TABLE table_from_csv ENGINE MergeTree ORDER BY () AS SELECT * FROM file('empty.csv') ", querySettings) + .get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 636); + Assert.assertTrue(e.getMessage().contains("You can specify the structure manually: (in file/uri /var/lib/clickhouse/user_files/empty.csv). (CANNOT_EXTRACT_TABLE_STRUCTURE)"), + "Unexpected error message: " + e.getMessage()); + } + + querySettings.serverSetting("unknown_setting", "1"); + try (QueryResponse response = client.query("CREATE TABLE table_from_csv AS SELECT * FROM file('empty.csv')", querySettings) + .get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 115); + Assert.assertTrue(e.getMessage().startsWith("Code: 115. DB::Exception: Setting unknown_setting is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings. (UNKNOWN_SETTING)"), + "Unexpected error message: " + e.getMessage()); + } + + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(e.getMessage(), e); + } + + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("non-existing-user") + .setPassword("nothing") + .compressServerResponse(serverCompression) + .useHttpCompression(useHttpCompression) + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), 516); + Assert.assertTrue(e.getMessage().startsWith("Code: 516. DB::Exception: non-existing-user: Authentication failed: password is incorrect, or there is no user with such name. (AUTHENTICATION_FAILED)"), + e.getMessage()); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } + + if (!isCloud()) { + Client.Builder clientBuilder = new Client.Builder() + .addEndpoint("http://" + server.getHost() + ":" + server.getPort() + "/some-path") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(serverCompression) + .useHttpCompression(useHttpCompression) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + + try (Client client = clientBuilder.build()) { + client.queryAll("select 1"); + fail("Exception expected"); + } catch (ClientException e) { + Assert.assertTrue(e.getCause().getMessage().contains("no handle /some-path?")); + } + } + } + + @DataProvider(name = "testServerErrorHandlingDataProvider") + public static Object[][] testServerErrorHandlingDataProvider() { + EnumSet formats = EnumSet.of(ClickHouseFormat.CSV, ClickHouseFormat.TSV, + ClickHouseFormat.JSON, ClickHouseFormat.JSONCompact); + + int permutations = 3; + Object[][] result = new Object[formats.size() * permutations][]; + + int i = 0; + for (ClickHouseFormat format : formats) { + // format, server compression, http compression + result[i++] = new Object[]{format, false, false}; + result[i++] = new Object[]{format, true, false}; + result[i++] = new Object[]{format, true, true}; + } + + return result; + } + + @Test(groups = { "integration" }) + public void testErrorWithSuccessfulResponse() { + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withChunkedDribbleDelay(2, 200) + .withHeader("X-ClickHouse-Exception-Code", "241") + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")) + .build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertTrue(e.getMessage().startsWith("Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB")); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }, dataProvider = "testServerErrorsUncompressedDataProvider") + public void testServerErrorsUncompressed(int code, String message, String expectedMessage) { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withChunkedDribbleDelay(2, 200) + .withHeader("X-ClickHouse-Exception-Code", String.valueOf(code)) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(message)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.fail("Expected exception"); + } catch (ServerException e) { + e.printStackTrace(); + Assert.assertEquals(e.getCode(), code); + Assert.assertTrue(e.getMessage().startsWith(expectedMessage), "but started with " + e.getMessage()); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testServerErrorsUncompressedDataProvider") + public static Object[][] testServerErrorsUncompressedDataProvider() { + return new Object[][] { + { 241, "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB", + "Code: 241. DB::Exception: Memory limit (for query) exceeded: would use 97.21 MiB"}, + {900, "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full", + "Code: 900. DB::Exception: \uD83D\uDCBE Floppy disk is full"}, + {901, "Code: 901. DB::Exception: I write, erase, rewrite\n" + + "Erase again, and then\n" + + "A poppy blooms\n" + + " (by Katsushika Hokusai)", + "Code: 901. DB::Exception: I write, erase, rewrite " + + "Erase again, and then " + + "A poppy blooms" + + " (by Katsushika Hokusai)"} + }; + } + + @Test(groups = { "integration" }) + public void testAdditionalHeaders() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .httpHeader("X-ClickHouse-Test", "default_value") + .httpHeader("X-ClickHouse-Test-2", Arrays.asList("default_value1", "default_value2")) + .httpHeader("X-ClickHouse-Test-3", Arrays.asList("default_value1", "default_value2")) + .httpHeader("X-ClickHouse-Test-4", "default_value4") + .build()) { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("X-ClickHouse-Test", WireMock.equalTo("test")) + .withHeader("X-ClickHouse-Test-2", WireMock.equalTo("test1,test2")) + .withHeader("X-ClickHouse-Test-3", WireMock.equalTo("default_value1,default_value2")) + .withHeader("X-ClickHouse-Test-4", WireMock.equalTo("default_value4")) + + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + QuerySettings querySettings = new QuerySettings() + .httpHeader("X-ClickHouse-Test", "test") + .httpHeader("X-ClickHouse-Test-2", Arrays.asList("test1", "test2")); + + try (QueryResponse response = client.query("SELECT 1", querySettings).get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testServerSettings() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .serverSetting("max_threads", "10") + .serverSetting("async_insert", "1") + .serverSetting("roles", Arrays.asList("role1", "role2")) + .compressClientRequest(true) + .build()) { + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withQueryParam("max_threads", WireMock.equalTo("10")) + .withQueryParam("async_insert", WireMock.equalTo("3")) + .withQueryParam("roles", WireMock.equalTo("role3,role2")) + .withQueryParam("compress", WireMock.equalTo("0")) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + QuerySettings querySettings = new QuerySettings() + .serverSetting("max_threads", "10") + .serverSetting("async_insert", "3") + .serverSetting("roles", Arrays.asList("role3", "role2")) + .serverSetting("compress", "0"); + try (QueryResponse response = client.query("SELECT 1", querySettings).get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail("Unexpected exception", e); + } finally { + mockServer.stop(); + } + } + } + + static { + if (Boolean.getBoolean("test.debug")) { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "DEBUG"); + } + } + + @Test(groups = { "integration" }) + public void testSSLAuthentication() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .setUsername("dba") + .setPassword("dba") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH ssl_certificate CN 'some_user'").get()) { + } + } + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .useSSLAuthentication(true) + .setUsername("some_user") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .setClientCertificate("some_user.crt") + .setClientKey("some_user.key") + .compressServerResponse(false) + .build()) { + + try (QueryResponse resp = client.query("SELECT 1").get()) { + Assert.assertEquals(resp.getReadRows(), 1); + } + } + } + + @Test(groups = { "integration" }, dataProvider = "testPasswordAuthenticationProvider", dataProviderClass = HttpTransportTests.class) + public void testPasswordAuthentication(String identifyWith, String identifyBy, boolean failsWithHeaders) throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("dba") + .setPassword("dba") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { + } + } catch (Exception e) { + Assert.fail("Failed on setup", e); + } + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .build()) { + + Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); + } catch (Exception e) { + Assert.fail("Failed to authenticate", e); + } + + if (failsWithHeaders) { + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .useHTTPBasicAuth(false) + .build()) { + + Assert.expectThrows(ClientException.class, () -> + client.queryAll("SELECT user()").get(0).getString(1)); + + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + } + + @DataProvider(name = "testPasswordAuthenticationProvider") + public static Object[][] testPasswordAuthenticationProvider() { + return new Object[][] { + { "plaintext_password", "password", false}, + { "plaintext_password", "", false }, + { "plaintext_password", "S3Cr=?t", true}, + { "plaintext_password", "123§", true }, + { "sha256_password", "password", false }, + { "sha256_password", "123§", true }, + { "sha256_password", "S3Cr=?t", true}, + { "sha256_password", "S3Cr?=t", false}, + }; + } + + @Test(groups = { "integration" }) + public void testAuthHeaderIsKeptFromUser() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + + String identifyWith = "sha256_password"; + String identifyBy = "123§"; + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("dba") + .setPassword("dba") + .build()) { + + try (CommandResponse resp = client.execute("DROP USER IF EXISTS some_user").get()) { + } + try (CommandResponse resp = client.execute("CREATE USER some_user IDENTIFIED WITH " + identifyWith + " BY '" + identifyBy + "'").get()) { + } + } catch (Exception e) { + Assert.fail("Failed on setup", e); + } + + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("some_user") + .setPassword(identifyBy) + .useHTTPBasicAuth(false) // disable basic auth to produce CH headers + .httpHeader(HttpHeaders.AUTHORIZATION, "Basic " + Base64.getEncoder().encodeToString(("some_user:" +identifyBy).getBytes())) + .build()) { + + Assert.assertEquals(client.queryAll("SELECT user()").get(0).getString(1), "some_user"); + } catch (Exception e) { + Assert.fail("Failed to authenticate", e); + } + } + + @Test(groups = { "integration" }) + public void testSSLAuthentication_invalidConfig() throws Exception { + if (isCloud()) { + return; // Current test is working only with local server because of self-signed certificates. + } + ClickHouseNode server = getSecureServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), true) + .useSSLAuthentication(true) + .setUsername("some_user") + .setPassword("s3cret") + .setRootCertificate("containers/clickhouse-server/certs/localhost.crt") + .setClientCertificate("some_user.crt") + .setClientKey("some_user.key") + .compressServerResponse(false) + .build()) { + fail("Expected exception"); + } catch (IllegalArgumentException e) { + e.printStackTrace(); + Assert.assertTrue(e.getMessage().startsWith("Only one of password, access token or SSL authentication")); + } + } + + @Test(groups = { "integration" }) + public void testErrorWithSendProgressHeaders() throws Exception { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost",server.getPort(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + + try (CommandResponse resp = client.execute("DROP TABLE IF EXISTS test_omm_table").get()) { + } + try (CommandResponse resp = client.execute("CREATE TABLE test_omm_table ( val String) Engine = MergeTree ORDER BY () ").get()) { + } + + QuerySettings settings = new QuerySettings() + .serverSetting("send_progress_in_http_headers", "1") + .serverSetting("max_memory_usage", "54M"); + + try (QueryResponse resp = client.query("INSERT INTO test_omm_table SELECT randomString(16) FROM numbers(300000000)", settings).get()) { + + } catch (ServerException e) { + // 241 - MEMORY_LIMIT_EXCEEDED or 243 -NOT_ENOUGH_SPACE + Assert.assertTrue(e.getCode() == 241 || e.getCode() == 243); + } + } + } + + + @Test(groups = { "integration" }, dataProvider = "testUserAgentHasCompleteProductName_dataProvider", dataProviderClass = HttpTransportTests.class) + public void testUserAgentHasCompleteProductName(String clientName, Pattern userAgentPattern) throws Exception { + if (isCloud()) { + return; // mocked server + } + + ClickHouseNode server = getServer(ClickHouseProtocol.HTTP); + try (Client client = new Client.Builder() + .addEndpoint(server.getBaseUri()) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setClientName(clientName) + .build()) { + + String q1Id = UUID.randomUUID().toString(); + + client.execute("SELECT 1", (CommandSettings) new CommandSettings().setQueryId(q1Id)).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT http_user_agent, http_referer, " + + " forwarded_for FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + q1Id + "'"); + Assert.assertFalse(logRecords.isEmpty(), "No records found in query log"); + + for (GenericRecord record : logRecords) { + System.out.println(record.getString("http_user_agent")); + Assert.assertTrue(userAgentPattern.matcher(record.getString("http_user_agent")).matches(), + record.getString("http_user_agent") + " doesn't match \"" + + userAgentPattern.pattern() + "\""); + + } + } + } + + + @DataProvider(name = "testUserAgentHasCompleteProductName_dataProvider") + public static Object[][] testUserAgentHasCompleteProductName_dataProvider() { + return new Object[][] { + { "", Pattern.compile("clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$") }, + { "test-client/1.0", Pattern.compile("test-client/1.0 clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}, + { "test-client/", Pattern.compile("test-client/ clickhouse-java-v2\\/.+ \\(.+\\) Apache-HttpClient\\/[\\d\\.]+$")}}; + } + + @Test(dataProvider = "testClientNameDataProvider") + public void testClientName(String clientName, boolean setWithUpdate, String userAgentHeader, boolean setForRequest) throws Exception { + + final String initialClientName = setWithUpdate ? "init clientName" : clientName; + final String initialUserAgentHeader = setForRequest ? "init userAgentHeader" : userAgentHeader; + final String clientReferer = "http://localhost/webpage"; + + Client.Builder builder = newClient(); + if (initialClientName != null) { + builder.setClientName(initialClientName); + } + if (initialUserAgentHeader != null) { + builder.httpHeader(HttpHeaders.USER_AGENT, initialUserAgentHeader); + } + try (Client client = builder.build()) { + String expectedClientNameStartsWith = initialClientName == null || initialUserAgentHeader != null ? initialUserAgentHeader : initialClientName; + + if (setWithUpdate) { + client.updateClientName(clientName); + expectedClientNameStartsWith = initialUserAgentHeader == null ? clientName : initialUserAgentHeader; + } + + String qId = UUID.randomUUID().toString(); + QuerySettings settings = new QuerySettings() + .httpHeader(HttpHeaders.REFERER, clientReferer) + .setQueryId(qId); + + if (setForRequest) { + settings.httpHeader(HttpHeaders.USER_AGENT, userAgentHeader); + expectedClientNameStartsWith = userAgentHeader; + } + + client.query("SELECT 1", settings).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + + " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); + Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); + final String logUserAgent = logRecords.get(0).getString("http_user_agent"); + Assert.assertTrue(logUserAgent.startsWith(expectedClientNameStartsWith), + "Expected to start with \"" + expectedClientNameStartsWith + "\" but values was \"" + logUserAgent + "\"" ); + Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); + Assert.assertEquals(logRecords.get(0).getString("http_referer"), clientReferer); + Assert.assertEquals(logRecords.get(0).getString("client_name"), ""); // http client can't set this field + } + } + + @DataProvider(name = "testClientNameDataProvider") + public static Object[][] testClientName() { + return new Object[][] { + {"test-product (app 1.0)", false, null, false}, // only client name set + {"test-product (app 1.0)", false, "final product (app 1.1)", false}, // http header set and overrides client name + {"test-product (app 1.0)", true, null, false}, // client name set thru Client#updateClientName + {"test-product (app 1.0)", true, "final product (app 1.1)", true}, // custom UserAgent header overrides client name + }; + } + + @Test(dataProvider = "testClientNameThruRawOptionsDataProvider") + public void testClientNameThruRawOptions(String property, String value, boolean setInClient) throws Exception { + Client.Builder builder = newClient(); + if (setInClient) { + builder.setOption(property, value); + } + try (Client client = builder.build()) { + + String qId = UUID.randomUUID().toString(); + QuerySettings settings = new QuerySettings() + .setQueryId(qId); + + if (!setInClient) { + settings.setOption(property, value); + } + + client.query("SELECT 1", settings).get().close(); + client.execute("SYSTEM FLUSH LOGS").get().close(); + + List logRecords = client.queryAll("SELECT query_id, client_name, http_user_agent, http_referer " + + " FROM clusterAllReplicas('default', system.query_log) WHERE query_id = '" + settings.getQueryId() + "'"); + Assert.assertEquals(logRecords.get(0).getString("query_id"), settings.getQueryId()); + final String logUserAgent = logRecords.get(0).getString("http_user_agent"); + Assert.assertTrue(logUserAgent.startsWith(value), + "Expected to start with \"" + value + "\" but values was \"" + logUserAgent + "\"" ); + Assert.assertTrue(logUserAgent.contains(Client.CLIENT_USER_AGENT), "Expected to contain client v2 version but value was \"" + logUserAgent + "\""); + } + } + + @DataProvider(name = "testClientNameThruRawOptionsDataProvider") + public Object[][] testClientNameThruRawOptionsDataProvider() { + return new Object[][] { + {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", true}, + {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", true}, + {ClientConfigProperties.PRODUCT_NAME.getKey(), "my product 1", false}, + {ClientConfigProperties.CLIENT_NAME.getKey(), "my product 2", false}, + }; + } + + @Test(groups = { "integration" }) + public void testBearerTokenAuth() throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + String jwtToken1 = Arrays.stream( + new String[]{"header", "payload", "signature"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) + .build()) { + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Unexpected exception", e); + } + } + + String jwtToken2 = Arrays.stream( + new String[]{"header2", "payload2", "signature2"}) + .map(s -> Base64.getEncoder().encodeToString(s.getBytes(StandardCharsets.UTF_8))) + .reduce((s1, s2) -> s1 + "." + s2).get(); + + mockServer.resetAll(); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken1)) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_UNAUTHORIZED)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .useBearerTokenAuth(jwtToken1) + .compressServerResponse(false) + .build()) { + + try { + client.execute("SELECT 1").get(); + fail("Exception expected"); + } catch (ServerException e) { + Assert.assertEquals(e.getTransportProtocolCode(), HttpStatus.SC_UNAUTHORIZED); + } + + mockServer.resetAll(); + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo("Bearer " + jwtToken2)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")) + + .build()); + + client.updateBearerToken(jwtToken2); + + client.execute("SELECT 1").get(); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testBasicAuthWithNoPassword() throws Exception { + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + // Expected: "default:" with empty password, not "default:null" + String expectedAuth = "Basic " + Base64.getEncoder() + .encodeToString("default:".getBytes(StandardCharsets.UTF_8)); + + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .withHeader("Authorization", WireMock.equalTo(expectedAuth)) + .willReturn(WireMock.aResponse() + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .compressServerResponse(false) + // no setPassword() call — password should default to empty, not "null" + .build()) { + + try (QueryResponse response = client.query("SELECT 1").get(1, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 10); + } catch (Exception e) { + Assert.fail("Basic auth with no password should send empty password, not 'null'", e); + } + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = { "integration" }) + public void testJWTWithCloud() throws Exception { + if (!isCloud()) { + return; // only for cloud + } + String jwt = System.getenv("CLIENT_JWT"); + Assert.assertTrue(jwt != null && !jwt.trim().isEmpty(), "JWT is missing"); + Assert.assertFalse(jwt.contains("\n") || jwt.contains("-----"), "JWT should be single string ready for HTTP header"); + try (Client client = newClient().useBearerTokenAuth(jwt).build()) { + try { + List response = client.queryAll("SELECT user(), now()"); + System.out.println("response: " + response.get(0).getString(1) + " time: " + response.get(0).getString(2)); + } catch (Exception e) { + e.printStackTrace(); + throw e; + } + } + } + + @Test(groups = { "integration" }) + public void testWithDefaultTimeouts() { + if (isCloud()) { + return; // mocked server + } + + int proxyPort = new Random().nextInt(1000) + 10000; + WireMockServer proxy = new WireMockServer(WireMockConfiguration + .options().port(proxyPort) + .notifier(new Slf4jNotifier(true))); + proxy.start(); + proxy.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withFixedDelay(5000) + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", proxyPort, false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .build()) { + int startTime = (int) System.currentTimeMillis(); + try { + client.query("SELECT 1").get(); + } catch (Exception e) { + Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); + } + } finally { + proxy.stop(); + } + } + + + @Test(groups = { "integration" }) + public void testTimeoutsWithRetry() { + if (isCloud()) { + return; // mocked server + } + + WireMockServer faultyServer = new WireMockServer( WireMockConfiguration + .options().port(9090).notifier(new ConsoleNotifier(false))); + faultyServer.start(); + + // First request gets no response + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .inScenario("Timeout") + .withRequestBody(WireMock.containing("SELECT 1")) + .whenScenarioStateIs(STARTED) + .willSetStateTo("Failed") + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withFixedDelay(5000) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + // Second request gets a response (retry) + faultyServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .inScenario("Timeout") + .withRequestBody(WireMock.containing("SELECT 1")) + .whenScenarioStateIs("Failed") + .willSetStateTo("Done") + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withFixedDelay(1000) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}")).build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", faultyServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setSocketTimeout(3000) + .retryOnFailures(ClientFaultCause.SocketTimeout) + .build()) { + int startTime = (int) System.currentTimeMillis(); + try { + client.query("SELECT 1").get(); + } catch (Exception e) { + Assert.fail("Elapsed Time: " + (System.currentTimeMillis() - startTime), e); + } + } finally { + faultyServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testSNIWithCloud() throws Exception { + if (!isCloud()) { + // skip for local env + return; + } + + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + String ip = InetAddress.getByName(node.getHost()).getHostAddress(); + try (Client c = new Client.Builder() + .addEndpoint(Protocol.HTTP, ip, node.getPort(), true) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .sslSocketSNI(node.getHost()).build()) { + c.execute("SELECT 1"); + } + } + + @Test(groups = {"integration"}) + public void testEndpointUrlPathIsPreserved() throws Exception { + if (isCloud()) { + return; // mocked server + } + + int serverPort = new Random().nextInt(1000) + 10000; + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().port(serverPort) + .notifier(new Slf4jNotifier(true))); + mockServer.start(); + + try { + // Setup stubs for two virtual ClickHouse instances behind a reverse proxy + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/sales/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"100\", \"read_rows\": \"10\"}")).build()); + + mockServer.addStubMapping(WireMock.post(WireMock.urlPathEqualTo("/billing/db")) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", + "{ \"read_bytes\": \"200\", \"read_rows\": \"20\"}")).build()); + + // Test sales virtual instance + try (Client salesClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/sales/db") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = salesClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 100); + } + } + + // Test billing virtual instance - also verify query parameters in URL are ignored + try (Client billingClient = new Client.Builder() + .addEndpoint("http://localhost:" + serverPort + "/billing/db?ignored_param=value") + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + try (QueryResponse response = billingClient.query("SELECT 1").get(10, TimeUnit.SECONDS)) { + Assert.assertEquals(response.getReadBytes(), 200); + } + + // Verify that ignored_param is not in the request URL + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db")) + .withoutQueryParam("ignored_param")); + } + + // Verify requests were made to the correct paths + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/sales/db"))); + mockServer.verify(WireMock.postRequestedFor(WireMock.urlPathEqualTo("/billing/db"))); + + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testMultiPartRequest() { + final Map params = new HashMap<>(); + params.put("database_name", "system"); + params.put("table_names", + DataTypeConverter.INSTANCE.arrayToString(Arrays.asList("COLLATIONS", "ENGINES"), "Array(String)")); + + // Use http compression + try (Client client = newClient().useHttpCompression(true).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + + // Use http compression + try (Client client = newClient().useHttpCompression(false).setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true").build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + + // compress request + try (Client client = newClient() + .compressClientRequest(true) + .setOption(ClientConfigProperties.HTTP_SEND_PARAMS_IN_BODY.getKey(), "true") + .useHttpCompression(true).build()) { + List records = client.queryAll("SELECT database, name FROM system.tables WHERE name IN {table_names:Array(String)}", + params); + + Assert.assertEquals(records.get(0).getString("name"), "COLLATIONS"); + Assert.assertEquals(records.get(1).getString("name"), "ENGINES"); + } + } + + @Test(groups = {"integration"}) + public void testSmallNetworkBufferDoesNotBreakColumnDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + Assert.assertTrue(validBody.length > 3, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - 5); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading truncated binary stream"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testChunkedResponsePrematureEndIsReported() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT 10"; + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader(HttpHeaders.TRANSFER_ENCODING, "chunked") + .withFault(Fault.MALFORMED_RESPONSE_CHUNK)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, null, 30, + "Expected failure when reading malformed chunked response"); + ConnectionClosedException connectionClosedException = findCause(thrown, ConnectionClosedException.class); + boolean hasChunkedPrematureCloseSignature = containsMessageInCauseChain(thrown, + "closing chunk expected", + "premature end of chunk coded message body", + "failed to read header"); + Assert.assertTrue(connectionClosedException != null || hasChunkedPrematureCloseSignature, + "Expected chunked/premature-close failure signature, but was: " + thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailCorruptedStreamFailsDecoding() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 100_000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + Throwable thrown = assertBinaryDecodeFails(mockServer, query, networkBufferSize, 60, + "Expected failure when reading binary stream truncated at tail"); + assertBinaryReadFailureContainsColumnName(thrown); + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}) + public void testTailStreamFailureReportsPositiveTimeSinceLastNextCall() throws Exception { + if (isCloud()) { + return; // mocked server + } + + final int rowsToRead = 2000; + final int networkBufferSize = 8196; + final String query = "SELECT toUInt32(number), toUInt64(number), now() FROM system.numbers LIMIT " + rowsToRead; + byte[] validBody = fetchBinaryPayload(query, networkBufferSize, 60); + final int removedBytes = 3; + Assert.assertTrue(validBody.length > removedBytes, "Source binary payload is unexpectedly small"); + byte[] corruptedBody = Arrays.copyOf(validBody, validBody.length - removedBytes); + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(HttpStatus.SC_OK) + .withHeader("X-ClickHouse-Summary", "{ \"read_bytes\": \"10\", \"read_rows\": \"1\"}") + .withBody(corruptedBody)) + .build()); + + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize) + .build(); + QueryResponse response = client.query(query, querySettings).get(60, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + final int[] rowsRead = new int[] {0}; + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + while (true) { + if (rowsRead[0] >= 5) { + try { + Thread.sleep(25); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + if (reader.next() == null) { + return; + } + rowsRead[0]++; + } + }); + + Assert.assertTrue(rowsRead[0] >= 5, + "Expected to read at least a few rows before failure, but read " + rowsRead[0]); + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + + String elapsedTimeMessage = findFirstMessageInCauseChain(thrown, "time since last next call"); + Assert.assertNotNull(elapsedTimeMessage, + "Expected elapsed-time fragment in failure message chain, but was: " + thrown); + + java.util.regex.Matcher matcher = Pattern.compile("time since last next call (\\d+)\\)") + .matcher(elapsedTimeMessage); + Assert.assertTrue(matcher.find(), + "Expected elapsed-time fragment in message: " + elapsedTimeMessage); + long elapsedSinceLastNext = Long.parseLong(matcher.group(1)); + Assert.assertTrue(elapsedSinceLastNext > 0, + "Expected positive elapsed time since last next call, but was " + elapsedSinceLastNext); + } + } finally { + mockServer.stop(); + } + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusErrorBodyDataProvider") + public void testHttpStatusErrorsIncludeResponseBody(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withBody(responseBody)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected to contain '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusErrorBodyDataProvider") + public static Object[][] testHttpStatusErrorBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: invalid credentials for user default", "invalid credentials"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: user default has no access to this operation", "no access"}, + {HttpStatus.SC_NOT_FOUND, "Not found: requested endpoint does not exist", "requested endpoint"} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusWithoutBodyDataProvider") + public void testHttpStatusErrorsWithoutBody(int httpStatus) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse().withStatus(httpStatus)) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressServerResponse(false) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains("unknown server error"), + "Expected unknown error message for empty body, but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusWithoutBodyDataProvider") + public static Object[][] testHttpStatusWithoutBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED}, + {HttpStatus.SC_FORBIDDEN}, + {HttpStatus.SC_NOT_FOUND} + }; + } + + @Test(groups = {"integration"}, dataProvider = "testHttpStatusCompressedBodyDataProvider") + public void testHttpStatusErrorsWithHttpCompression(int httpStatus, String responseBody, String expectedBodyPart) throws Exception { + if (isCloud()) { + return; // mocked server + } + + WireMockServer mockServer = new WireMockServer(WireMockConfiguration + .options().dynamicPort().notifier(new ConsoleNotifier(false))); + mockServer.start(); + + try { + mockServer.addStubMapping(WireMock.post(WireMock.anyUrl()) + .willReturn(WireMock.aResponse() + .withStatus(httpStatus) + .withHeader(HttpHeaders.CONTENT_ENCODING, "gzip") + .withBody(gzip(responseBody))) + .build()); + + try (Client client = new Client.Builder().addEndpoint(Protocol.HTTP, "localhost", mockServer.port(), false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .useHttpCompression(true) + .compressServerResponse(true) + .build()) { + + Throwable thrown = Assert.expectThrows(Throwable.class, + () -> client.query("SELECT 1").get(1, TimeUnit.SECONDS)); + ServerException serverException = findServerException(thrown); + Assert.assertNotNull(serverException, "Expected ServerException in cause chain"); + Assert.assertEquals(serverException.getTransportProtocolCode(), httpStatus); + Assert.assertTrue(serverException.getMessage().contains(expectedBodyPart), + "Expected compressed body part '" + expectedBodyPart + "', but was: " + serverException.getMessage()); + } + } finally { + mockServer.stop(); + } + } + + @DataProvider(name = "testHttpStatusCompressedBodyDataProvider") + public static Object[][] testHttpStatusCompressedBodyDataProvider() { + return new Object[][]{ + {HttpStatus.SC_UNAUTHORIZED, "Unauthorized: token is expired", "token is expired"}, + {HttpStatus.SC_FORBIDDEN, "Forbidden: policy denies this query", "policy denies"}, + {HttpStatus.SC_NOT_FOUND, "Not found: route does not exist", "route does not exist"} + }; + } + + private byte[] fetchBinaryPayload(String query, int networkBufferSize, int timeoutSec) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newClient() + .useHttpCompression(false) + .compressServerResponse(false) + .setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)) + .build(); + QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS)) { + return readAllBytes(response.getInputStream()); + } + } + + private Throwable assertBinaryDecodeFails(WireMockServer mockServer, String query, Integer networkBufferSize, + int timeoutSec, String assertionMessage) throws Exception { + QuerySettings querySettings = new QuerySettings().setFormat(ClickHouseFormat.RowBinaryWithNamesAndTypes); + try (Client client = newMockServerClient(mockServer.port(), networkBufferSize).build()) { + Throwable thrown = Assert.expectThrows(Throwable.class, () -> { + try (QueryResponse response = client.query(query, querySettings).get(timeoutSec, TimeUnit.SECONDS); + ClickHouseBinaryFormatReader reader = client.newBinaryFormatReader(response)) { + readAllRows(reader); + } + }); + Assert.assertNotNull(thrown, assertionMessage); + return thrown; + } + } + + private Client.Builder newMockServerClient(int port, Integer networkBufferSize) { + Client.Builder builder = new Client.Builder() + .addEndpoint(Protocol.HTTP, "localhost", port, false) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1") + .useHttpCompression(false) + .compressServerResponse(false); + if (networkBufferSize != null) { + builder.setOption(ClientConfigProperties.CLIENT_NETWORK_BUFFER_SIZE.getKey(), String.valueOf(networkBufferSize)); + } + return builder; + } + + private static void readAllRows(ClickHouseBinaryFormatReader reader) { + while (reader.next() != null) { + reader.getInteger(1); + reader.getLong(2); + reader.getString(3); + } + } + + private static byte[] gzip(String value) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(out)) { + gzipOutputStream.write(value.getBytes(StandardCharsets.UTF_8)); + } + return out.toByteArray(); + } + + private static byte[] readAllBytes(java.io.InputStream inputStream) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + byte[] buffer = new byte[8192]; + int read; + while ((read = inputStream.read(buffer)) != -1) { + out.write(buffer, 0, read); + } + return out.toByteArray(); + } + + private static ServerException findServerException(Throwable throwable) { + Throwable current = throwable; + while (current != null) { + if (current instanceof ServerException) { + return (ServerException) current; + } + current = current.getCause(); + } + return null; + } + + private static T findCause(Throwable throwable, Class clazz) { + Throwable current = throwable; + while (current != null) { + if (clazz.isInstance(current)) { + return clazz.cast(current); + } + current = current.getCause(); + } + return null; + } + + private static boolean containsMessageInCauseChain(Throwable throwable, String... parts) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null) { + String lower = message.toLowerCase(); + for (String part : parts) { + if (lower.contains(part.toLowerCase())) { + return true; + } + } + } + current = current.getCause(); + } + return false; + } + + private static String findFirstMessageInCauseChain(Throwable throwable, String part) { + Throwable current = throwable; + while (current != null) { + String message = current.getMessage(); + if (message != null && message.contains(part)) { + return message; + } + current = current.getCause(); + } + return null; + } + + private static void assertBinaryReadFailureContainsColumnName(Throwable thrown) { + ClientException clientException = findCause(thrown, ClientException.class); + Assert.assertNotNull(clientException, + "Expected ClientException in cause chain, but was: " + thrown); + Assert.assertTrue(containsMessageInCauseChain(thrown, "Reading column "), + "Expected column information in failure message chain, but was: " + thrown); + } + + protected Client.Builder newClient() { + ClickHouseNode node = getServer(ClickHouseProtocol.HTTP); + boolean isSecure = isCloud(); + return new Client.Builder() + .addEndpoint(Protocol.HTTP, node.getHost(), node.getPort(), isSecure) + .setUsername("default") + .setPassword(ClickHouseServerForTest.getPassword()) + .compressClientRequest(false) + .setDefaultDatabase(ClickHouseServerForTest.getDatabase()) + .serverSetting(ServerSettings.WAIT_END_OF_QUERY, "1"); + } +} From e3fd015cdc06952008de6eb6a1387ea468a4ea59 Mon Sep 17 00:00:00 2001 From: Sergey Chernov Date: Thu, 2 Apr 2026 15:21:52 -0700 Subject: [PATCH 6/6] fixed reading error message from compressed --- .../main/java/com/clickhouse/client/api/internal/LZ4Entity.java | 1 - 1 file changed, 1 deletion(-) diff --git a/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java b/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java index 8d765a130..1cd75c7e3 100644 --- a/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java +++ b/client-v2/src/main/java/com/clickhouse/client/api/internal/LZ4Entity.java @@ -7,7 +7,6 @@ import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpEntity; -import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream;