@@ -1204,14 +1204,17 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12041204 settings = new InsertSettings ();
12051205 }
12061206
1207+ final InsertSettings requestSettings = new InsertSettings (buildRequestSettings (settings .getAllSettings ()));
1208+
12071209 String operationId = registerOperationMetrics ();
1208- settings .setOperationId (operationId );
1210+ requestSettings .setOperationId (operationId );
12091211 globalClientStats .get (operationId ).start (ClientMetrics .OP_DURATION );
12101212 globalClientStats .get (operationId ).start (ClientMetrics .OP_SERIALIZATION );
12111213
12121214
12131215 boolean hasDefaults = this .tableSchemaHasDefaults .get (tableName );
12141216 ClickHouseFormat format = hasDefaults ? ClickHouseFormat .RowBinaryWithDefaults : ClickHouseFormat .RowBinary ;
1217+ requestSettings .setOption (ClientConfigProperties .INPUT_OUTPUT_FORMAT .getKey (), format );
12151218 TableSchema tableSchema = tableSchemaCache .get (tableName );
12161219 if (tableSchema == null ) {
12171220 throw new IllegalArgumentException ("Table schema not found for table: " + tableName + ". Did you forget to register it?" );
@@ -1235,8 +1238,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12351238 Integer retry = (Integer ) configuration .get (ClientConfigProperties .RETRY_ON_FAILURE .getKey ());
12361239 final int maxRetries = retry == null ? 0 : retry ;
12371240
1238- settings .setOption (ClientConfigProperties .INPUT_OUTPUT_FORMAT .getKey (), format );
1239- final InsertSettings finalSettings = new InsertSettings (buildRequestSettings (settings .getAllSettings ()));
1241+ requestSettings .setOption (ClientConfigProperties .INPUT_OUTPUT_FORMAT .getKey (), format );
12401242 Supplier <InsertResponse > supplier = () -> {
12411243 long startTime = System .nanoTime ();
12421244 // Selecting some node
@@ -1246,7 +1248,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12461248 for (int i = 0 ; i <= maxRetries ; i ++) {
12471249 // Execute request
12481250 try (ClassicHttpResponse httpResponse =
1249- httpClientHelper .executeRequest (selectedEndpoint , finalSettings .getAllSettings (), lz4Factory ,
1251+ httpClientHelper .executeRequest (selectedEndpoint , requestSettings .getAllSettings (), lz4Factory ,
12501252 out -> {
12511253 out .write ("INSERT INTO " .getBytes ());
12521254 out .write (tableName .getBytes ());
@@ -1278,14 +1280,14 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12781280 OperationMetrics metrics = new OperationMetrics (clientStats );
12791281 String summary = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_SRV_SUMMARY ), "{}" );
12801282 ProcessParser .parseSummary (summary , metrics );
1281- String queryId = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), finalSettings .getQueryId (), String ::valueOf );
1283+ String queryId = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), requestSettings .getQueryId (), String ::valueOf );
12821284 metrics .operationComplete ();
12831285 metrics .setQueryId (queryId );
12841286 return new InsertResponse (metrics );
12851287 } catch (Exception e ) {
12861288 lastException = httpClientHelper .wrapException (String .format ("Query request failed (Attempt: %s/%s - Duration: %s)" ,
12871289 (i + 1 ), (maxRetries + 1 ), System .nanoTime () - startTime ), e );
1288- if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1290+ if (httpClientHelper .shouldRetry (e , requestSettings .getAllSettings ())) {
12891291 LOG .warn ("Retrying." , e );
12901292 selectedEndpoint = getNextAliveNode ();
12911293 } else {
@@ -1296,7 +1298,7 @@ public CompletableFuture<InsertResponse> insert(String tableName, List<?> data,
12961298 throw new ClientException ("Insert request failed after attempts: " + (maxRetries + 1 ) + " - Duration: " + (System .nanoTime () - startTime ), lastException );
12971299 };
12981300
1299- return runAsyncOperation (supplier , settings .getAllSettings ());
1301+ return runAsyncOperation (supplier , requestSettings .getAllSettings ());
13001302
13011303 }
13021304
@@ -1415,8 +1417,13 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14151417 DataStreamWriter writer ,
14161418 ClickHouseFormat format ,
14171419 InsertSettings settings ) {
1420+ if (settings == null ) {
1421+ throw new IllegalArgumentException ("Settings cannot be null" );
1422+ }
1423+ final InsertSettings requestSettings = new InsertSettings (buildRequestSettings (settings .getAllSettings ()));
1424+ requestSettings .setOption (ClientConfigProperties .INPUT_OUTPUT_FORMAT .getKey (), format );
14181425
1419- String operationId = settings .getOperationId ();
1426+ String operationId = requestSettings .getOperationId ();
14201427 ClientStatisticsHolder clientStats = null ;
14211428 if (operationId != null ) {
14221429 clientStats = globalClientStats .remove (operationId );
@@ -1430,17 +1437,14 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14301437
14311438 Supplier <InsertResponse > responseSupplier ;
14321439
1433- final int writeBufferSize = settings .getInputStreamCopyBufferSize () <= 0 ?
1440+ final int writeBufferSize = requestSettings .getInputStreamCopyBufferSize () <= 0 ?
14341441 (int ) configuration .get (ClientConfigProperties .CLIENT_NETWORK_BUFFER_SIZE .getKey ()) :
1435- settings .getInputStreamCopyBufferSize ();
1442+ requestSettings .getInputStreamCopyBufferSize ();
14361443
14371444 if (writeBufferSize <= 0 ) {
14381445 throw new IllegalArgumentException ("Buffer size must be greater than 0" );
14391446 }
14401447
1441- settings .setOption (ClientConfigProperties .INPUT_OUTPUT_FORMAT .getKey (), format );
1442- final InsertSettings finalSettings = new InsertSettings (buildRequestSettings (settings .getAllSettings ()));
1443-
14441448 StringBuilder sqlStmt = new StringBuilder ("INSERT INTO " ).append (tableName );
14451449 if (columnNames != null && !columnNames .isEmpty ()) {
14461450 sqlStmt .append (" (" );
@@ -1451,7 +1455,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14511455 sqlStmt .append (")" );
14521456 }
14531457 sqlStmt .append (" FORMAT " ).append (format .name ());
1454- finalSettings .serverSetting (ClickHouseHttpProto .QPARAM_QUERY_STMT , sqlStmt .toString ());
1458+ requestSettings .serverSetting (ClickHouseHttpProto .QPARAM_QUERY_STMT , sqlStmt .toString ());
14551459 responseSupplier = () -> {
14561460 long startTime = System .nanoTime ();
14571461 // Selecting some node
@@ -1461,7 +1465,7 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14611465 for (int i = 0 ; i <= retries ; i ++) {
14621466 // Execute request
14631467 try (ClassicHttpResponse httpResponse =
1464- httpClientHelper .executeRequest (selectedEndpoint , finalSettings .getAllSettings (), lz4Factory ,
1468+ httpClientHelper .executeRequest (selectedEndpoint , requestSettings .getAllSettings (), lz4Factory ,
14651469 out -> {
14661470 writer .onOutput (out );
14671471 out .close ();
@@ -1478,14 +1482,14 @@ public CompletableFuture<InsertResponse> insert(String tableName,
14781482 OperationMetrics metrics = new OperationMetrics (finalClientStats );
14791483 String summary = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_SRV_SUMMARY ), "{}" );
14801484 ProcessParser .parseSummary (summary , metrics );
1481- String queryId = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), finalSettings .getQueryId (), String ::valueOf );
1485+ String queryId = HttpAPIClientHelper .getHeaderVal (httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), requestSettings .getQueryId (), String ::valueOf );
14821486 metrics .operationComplete ();
14831487 metrics .setQueryId (queryId );
14841488 return new InsertResponse (metrics );
14851489 } catch (Exception e ) {
14861490 lastException = httpClientHelper .wrapException (String .format ("Insert failed (Attempt: %s/%s - Duration: %s)" ,
14871491 (i + 1 ), (retries + 1 ), System .nanoTime () - startTime ), e );
1488- if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1492+ if (httpClientHelper .shouldRetry (e , requestSettings .getAllSettings ())) {
14891493 LOG .warn ("Retrying." , e );
14901494 selectedEndpoint = getNextAliveNode ();
14911495 } else {
@@ -1502,10 +1506,10 @@ public CompletableFuture<InsertResponse> insert(String tableName,
15021506 }
15031507 }
15041508 LOG .warn ("Insert request failed after attempts: " + (retries + 1 ) + " - Duration: " + (System .nanoTime () - startTime ));
1505- throw lastException ;
1509+ throw ( lastException == null ? new ClientException ( "Failed to complete insert operation" ) : lastException ) ;
15061510 };
15071511
1508- return runAsyncOperation (responseSupplier , settings .getAllSettings ());
1512+ return runAsyncOperation (responseSupplier , requestSettings .getAllSettings ());
15091513 }
15101514
15111515 /**
@@ -1564,18 +1568,19 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15641568 if (settings == null ) {
15651569 settings = new QuerySettings ();
15661570 }
1567- if (settings .getFormat () == null ) {
1568- settings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
1571+ final QuerySettings requestSettings = new QuerySettings (buildRequestSettings (settings .getAllSettings ()));
1572+
1573+ if (requestSettings .getFormat () == null ) {
1574+ requestSettings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
15691575 }
15701576 ClientStatisticsHolder clientStats = new ClientStatisticsHolder ();
15711577 clientStats .start (ClientMetrics .OP_DURATION );
15721578
15731579 Supplier <QueryResponse > responseSupplier ;
15741580
15751581 if (queryParams != null ) {
1576- settings .setOption (HttpAPIClientHelper .KEY_STATEMENT_PARAMS , queryParams );
1582+ requestSettings .setOption (HttpAPIClientHelper .KEY_STATEMENT_PARAMS , queryParams );
15771583 }
1578- final QuerySettings finalSettings = new QuerySettings (buildRequestSettings (settings .getAllSettings ()));
15791584 responseSupplier = () -> {
15801585 long startTime = System .nanoTime ();
15811586 // Selecting some node
@@ -1585,7 +1590,7 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
15851590 ClassicHttpResponse httpResponse = null ;
15861591 try {
15871592 httpResponse =
1588- httpClientHelper .executeRequest (selectedEndpoint , finalSettings .getAllSettings (), lz4Factory , output -> {
1593+ httpClientHelper .executeRequest (selectedEndpoint , requestSettings .getAllSettings (), lz4Factory , output -> {
15891594 output .write (sqlQuery .getBytes (StandardCharsets .UTF_8 ));
15901595 output .close ();
15911596 });
@@ -1602,22 +1607,22 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16021607 .getFirstHeader (ClickHouseHttpProto .HEADER_SRV_SUMMARY ), "{}" );
16031608 ProcessParser .parseSummary (summary , metrics );
16041609 String queryId = HttpAPIClientHelper .getHeaderVal (httpResponse
1605- .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), finalSettings .getQueryId ());
1610+ .getFirstHeader (ClickHouseHttpProto .HEADER_QUERY_ID ), requestSettings .getQueryId ());
16061611 metrics .setQueryId (queryId );
16071612 metrics .operationComplete ();
16081613 Header formatHeader = httpResponse .getFirstHeader (ClickHouseHttpProto .HEADER_FORMAT );
1609- ClickHouseFormat responseFormat = finalSettings .getFormat ();
1614+ ClickHouseFormat responseFormat = requestSettings .getFormat ();
16101615 if (formatHeader != null ) {
16111616 responseFormat = ClickHouseFormat .valueOf (formatHeader .getValue ());
16121617 }
16131618
1614- return new QueryResponse (httpResponse , responseFormat , finalSettings , metrics );
1619+ return new QueryResponse (httpResponse , responseFormat , requestSettings , metrics );
16151620
16161621 } catch (Exception e ) {
16171622 httpClientHelper .closeQuietly (httpResponse );
16181623 lastException = httpClientHelper .wrapException (String .format ("Query request failed (Attempt: %s/%s - Duration: %s)" ,
16191624 (i + 1 ), (retries + 1 ), System .nanoTime () - startTime ), e );
1620- if (httpClientHelper .shouldRetry (e , finalSettings .getAllSettings ())) {
1625+ if (httpClientHelper .shouldRetry (e , requestSettings .getAllSettings ())) {
16211626 LOG .warn ("Retrying." , e );
16221627 selectedEndpoint = getNextAliveNode ();
16231628 } else {
@@ -1626,10 +1631,10 @@ public CompletableFuture<QueryResponse> query(String sqlQuery, Map<String, Objec
16261631 }
16271632 }
16281633 LOG .warn ("Query request failed after attempts: " + (retries + 1 ) + " - Duration: " + (System .nanoTime () - startTime ));
1629- throw lastException ;
1634+ throw ( lastException == null ? new ClientException ( "Failed to complete query" ) : lastException ) ;
16301635 };
16311636
1632- return runAsyncOperation (responseSupplier , settings .getAllSettings ());
1637+ return runAsyncOperation (responseSupplier , requestSettings .getAllSettings ());
16331638 }
16341639 public CompletableFuture <QueryResponse > query (String sqlQuery , Map <String , Object > queryParams ) {
16351640 return query (sqlQuery , queryParams , null );
@@ -1674,10 +1679,11 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, Map<String, Obje
16741679 if (settings == null ) {
16751680 settings = new QuerySettings ();
16761681 }
1677- settings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
1678- settings .waitEndOfQuery (true ); // we rely on the summery
1682+ final QuerySettings requestSettings = new QuerySettings (buildRequestSettings (settings .getAllSettings ()));
1683+ requestSettings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
1684+ requestSettings .waitEndOfQuery (true ); // we rely on the summery
16791685
1680- return query (sqlQuery , params , settings ).thenApply (response -> {
1686+ return query (sqlQuery , params , requestSettings ).thenApply (response -> {
16811687 try {
16821688
16831689 return new Records (response , newBinaryFormatReader (response ));
@@ -1708,9 +1714,11 @@ public List<GenericRecord> queryAll(String sqlQuery, Map<String, Object> params,
17081714 }
17091715 try {
17101716 int operationTimeout = getOperationTimeout ();
1711- settings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes )
1712- .waitEndOfQuery (true );
1713- CompletableFuture <QueryResponse > f = query (sqlQuery , params , settings );
1717+ final QuerySettings requestSettings = new QuerySettings (buildRequestSettings (settings .getAllSettings ()));
1718+ requestSettings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
1719+ requestSettings .waitEndOfQuery (true );
1720+
1721+ CompletableFuture <QueryResponse > f = query (sqlQuery , params , requestSettings );
17141722 try (QueryResponse response = operationTimeout == 0 ? f .get () : f .get (operationTimeout , TimeUnit .MILLISECONDS )) {
17151723 List <GenericRecord > records = new ArrayList <>();
17161724 if (response .getResultRows () > 0 ) {
0 commit comments