1111import com .clickhouse .client .api .data_formats .RowBinaryFormatReader ;
1212import com .clickhouse .client .api .data_formats .RowBinaryWithNamesAndTypesFormatReader ;
1313import com .clickhouse .client .api .data_formats .RowBinaryWithNamesFormatReader ;
14+ import com .clickhouse .client .api .data_formats .internal .BinaryStreamReader ;
1415import com .clickhouse .client .api .data_formats .internal .MapBackedRecord ;
1516import com .clickhouse .client .api .data_formats .internal .ProcessParser ;
1617import com .clickhouse .client .api .enums .Protocol ;
1920import com .clickhouse .client .api .insert .InsertResponse ;
2021import com .clickhouse .client .api .insert .InsertSettings ;
2122import com .clickhouse .client .api .insert .POJOSerializer ;
23+ import com .clickhouse .client .api .insert .SerializerNotFoundException ;
24+ import com .clickhouse .client .api .internal .BasicObjectsPool ;
2225import com .clickhouse .client .api .internal .ClickHouseLZ4OutputStream ;
2326import com .clickhouse .client .api .internal .ClientStatisticsHolder ;
2427import com .clickhouse .client .api .internal .ClientV1AdaptorHelper ;
7073import java .util .HashMap ;
7174import java .util .HashSet ;
7275import java .util .LinkedHashMap ;
76+ import java .util .LinkedList ;
7377import java .util .List ;
7478import java .util .Map ;
7579import java .util .Set ;
@@ -761,6 +765,22 @@ public Builder setMaxRetries(int maxRetries) {
761765 return this ;
762766 }
763767
768+ /**
769+ * Configures client to reuse allocated byte buffers for numbers. It affects how binary format reader is working.
770+ * If set to 'true' then {@link Client#newBinaryFormatReader(QueryResponse)} will construct reader that will
771+ * reuse buffers for numbers. It improves performance for large datasets by reducing number of allocations
772+ * (therefore GC pressure).
773+ * Enabling this feature is safe because each reader suppose to be used by a single thread and readers are not reused.
774+ *
775+ * Default is false.
776+ * @param reuse - if to reuse buffers
777+ * @return
778+ */
779+ public Builder allowBinaryReaderToReuseBuffers (boolean reuse ) {
780+ this .configuration .put ("client_allow_binary_reader_to_reuse_buffers" , String .valueOf (reuse ));
781+ return this ;
782+ }
783+
764784 public Client build () {
765785 setDefaults ();
766786
@@ -874,6 +894,10 @@ private void setDefaults() {
874894 if (!configuration .containsKey (ClickHouseClientOption .RETRY .getKey ())) {
875895 setMaxRetries (3 );
876896 }
897+
898+ if (!configuration .containsKey ("client_allow_binary_reader_to_reuse_buffers" )) {
899+ allowBinaryReaderToReuseBuffers (false );
900+ }
877901 }
878902 }
879903
@@ -1468,10 +1492,10 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings se
14681492 settings .setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes );
14691493 settings .waitEndOfQuery (true ); // we rely on the summery
14701494
1471- final QuerySettings finalSettings = settings ;
14721495 return query (sqlQuery , settings ).thenApply (response -> {
14731496 try {
1474- return new Records (response , finalSettings );
1497+
1498+ return new Records (response , newBinaryFormatReader (response ));
14751499 } catch (Exception e ) {
14761500 throw new ClientException ("Failed to get query response" , e );
14771501 }
@@ -1488,13 +1512,14 @@ public CompletableFuture<Records> queryRecords(String sqlQuery, QuerySettings se
14881512 public List <GenericRecord > queryAll (String sqlQuery ) {
14891513 try {
14901514 int operationTimeout = getOperationTimeout ();
1491- QuerySettings settings = new QuerySettings ().waitEndOfQuery (true );
1515+ QuerySettings settings = new QuerySettings ().setFormat (ClickHouseFormat .RowBinaryWithNamesAndTypes )
1516+ .waitEndOfQuery (true );
14921517 try (QueryResponse response = operationTimeout == 0 ? query (sqlQuery , settings ).get () :
14931518 query (sqlQuery , settings ).get (operationTimeout , TimeUnit .MILLISECONDS )) {
14941519 List <GenericRecord > records = new ArrayList <>();
14951520 if (response .getResultRows () > 0 ) {
14961521 RowBinaryWithNamesAndTypesFormatReader reader =
1497- new RowBinaryWithNamesAndTypesFormatReader ( response . getInputStream (), response . getSettings () );
1522+ ( RowBinaryWithNamesAndTypesFormatReader ) newBinaryFormatReader ( response );
14981523
14991524 Map <String , Object > record ;
15001525 while (reader .readRecord ((record = new LinkedHashMap <>()))) {
@@ -1662,28 +1687,38 @@ public CompletableFuture<CommandResponse> execute(String sql) {
16621687 * @param schema
16631688 * @return
16641689 */
1665- public static ClickHouseBinaryFormatReader newBinaryFormatReader (QueryResponse response , TableSchema schema ) {
1690+ public ClickHouseBinaryFormatReader newBinaryFormatReader (QueryResponse response , TableSchema schema ) {
16661691 ClickHouseBinaryFormatReader reader = null ;
1692+ // Using caching buffer allocator is risky so this parameter is not exposed to the user
1693+ boolean useCachingBufferAllocator = MapUtils .getFlag (configuration , "client_allow_binary_reader_to_reuse_buffers" );
1694+ BinaryStreamReader .ByteBufferAllocator byteBufferPool = useCachingBufferAllocator ?
1695+ new BinaryStreamReader .CachingByteBufferAllocator () :
1696+ new BinaryStreamReader .DefaultByteBufferAllocator ();
1697+
16671698 switch (response .getFormat ()) {
16681699 case Native :
1669- reader = new NativeFormatReader (response .getInputStream (), response .getSettings ());
1700+ reader = new NativeFormatReader (response .getInputStream (), response .getSettings (),
1701+ byteBufferPool );
16701702 break ;
16711703 case RowBinaryWithNamesAndTypes :
1672- reader = new RowBinaryWithNamesAndTypesFormatReader (response .getInputStream (), response .getSettings ());
1704+ reader = new RowBinaryWithNamesAndTypesFormatReader (response .getInputStream (), response .getSettings (),
1705+ byteBufferPool );
16731706 break ;
16741707 case RowBinaryWithNames :
1675- reader = new RowBinaryWithNamesFormatReader (response .getInputStream (), response .getSettings (), schema );
1708+ reader = new RowBinaryWithNamesFormatReader (response .getInputStream (), response .getSettings (), schema ,
1709+ byteBufferPool );
16761710 break ;
16771711 case RowBinary :
1678- reader = new RowBinaryFormatReader (response .getInputStream (), response .getSettings (), schema );
1712+ reader = new RowBinaryFormatReader (response .getInputStream (), response .getSettings (), schema ,
1713+ byteBufferPool );
16791714 break ;
16801715 default :
16811716 throw new IllegalArgumentException ("Unsupported format: " + response .getFormat ());
16821717 }
16831718 return reader ;
16841719 }
16851720
1686- public static ClickHouseBinaryFormatReader newBinaryFormatReader (QueryResponse response ) {
1721+ public ClickHouseBinaryFormatReader newBinaryFormatReader (QueryResponse response ) {
16871722 return newBinaryFormatReader (response , null );
16881723 }
16891724
0 commit comments