26
26
import com .clickhouse .client .http .config .ClickHouseHttpOption ;
27
27
28
28
public abstract class ClickHouseHttpConnection implements AutoCloseable {
29
+ private static StringBuilder appendQueryParameter (StringBuilder builder , String key , String value ) {
30
+ return builder .append (urlEncode (key , StandardCharsets .UTF_8 )).append ('=' )
31
+ .append (urlEncode (value , StandardCharsets .UTF_8 )).append ('&' );
32
+ }
33
+
29
34
static String urlEncode (String str , Charset charset ) {
30
35
if (charset == null ) {
31
36
charset = StandardCharsets .UTF_8 ;
@@ -39,11 +44,6 @@ static String urlEncode(String str, Charset charset) {
39
44
}
40
45
}
41
46
42
- private static StringBuilder appendQueryParameter (StringBuilder builder , String key , String value ) {
43
- return builder .append (urlEncode (key , StandardCharsets .UTF_8 )).append ('=' )
44
- .append (urlEncode (value , StandardCharsets .UTF_8 )).append ('&' );
45
- }
46
-
47
47
static String buildQueryParams (ClickHouseRequest <?> request ) {
48
48
if (request == null ) {
49
49
return "" ;
@@ -146,26 +146,7 @@ static String buildUrl(String baseUrl, ClickHouseRequest<?> request) {
146
146
return builder .toString ();
147
147
}
148
148
149
- protected final ClickHouseConfig config ;
150
- protected final ClickHouseNode server ;
151
- protected final Map <String , String > defaultHeaders ;
152
-
153
- protected final ClickHouseOutputStream output ;
154
-
155
- protected final String url ;
156
-
157
- protected ClickHouseHttpConnection (ClickHouseNode server , ClickHouseRequest <?> request ) {
158
- if (server == null || request == null ) {
159
- throw new IllegalArgumentException ("Non-null server and request are required" );
160
- }
161
-
162
- this .config = request .getConfig ();
163
- this .server = server ;
164
-
165
- this .output = request .getOutputStream ().orElse (null );
166
-
167
- this .url = buildUrl (server .getBaseUri (), request );
168
-
149
+ protected static Map <String , String > createDefaultHeaders (ClickHouseConfig config , ClickHouseNode server ) {
169
150
Map <String , String > map = new LinkedHashMap <>();
170
151
// add customer headers
171
152
map .putAll (ClickHouseUtils .getKeyValuePairs ((String ) config .getOption (ClickHouseHttpOption .CUSTOM_HEADERS )));
@@ -201,8 +182,25 @@ protected ClickHouseHttpConnection(ClickHouseNode server, ClickHouseRequest<?> r
201
182
&& config .getRequestCompressAlgorithm () != ClickHouseCompression .LZ4 ) {
202
183
map .put ("Content-Encoding" , config .getRequestCompressAlgorithm ().encoding ());
203
184
}
185
+ return map ;
186
+ }
204
187
205
- this .defaultHeaders = Collections .unmodifiableMap (map );
188
+ protected final ClickHouseConfig config ;
189
+ protected final ClickHouseNode server ;
190
+ protected final ClickHouseOutputStream output ;
191
+ protected final String url ;
192
+ protected final Map <String , String > defaultHeaders ;
193
+
194
+ protected ClickHouseHttpConnection (ClickHouseNode server , ClickHouseRequest <?> request ) {
195
+ if (server == null || request == null ) {
196
+ throw new IllegalArgumentException ("Non-null server and request are required" );
197
+ }
198
+
199
+ this .config = request .getConfig ();
200
+ this .server = server ;
201
+ this .output = request .getOutputStream ().orElse (null );
202
+ this .url = buildUrl (server .getBaseUri (), request );
203
+ this .defaultHeaders = Collections .unmodifiableMap (createDefaultHeaders (config , server ));
206
204
}
207
205
208
206
protected void closeQuietly () {
@@ -231,11 +229,13 @@ protected String getBaseUrl() {
231
229
* Creates a merged map.
232
230
*
233
231
* @param requestHeaders request headers
234
- * @return
232
+ * @return non-null merged headers
235
233
*/
236
234
protected Map <String , String > mergeHeaders (Map <String , String > requestHeaders ) {
237
235
if (requestHeaders == null || requestHeaders .isEmpty ()) {
238
236
return defaultHeaders ;
237
+ } else if (isReusable ()) {
238
+ return requestHeaders ;
239
239
}
240
240
241
241
Map <String , String > merged = new LinkedHashMap <>();
@@ -256,13 +256,15 @@ protected Map<String, String> mergeHeaders(Map<String, String> requestHeaders) {
256
256
* @param query non-blank query
257
257
* @param data optionally input stream for batch updating
258
258
* @param tables optionally external tables for query
259
+ * @param url optionally url
259
260
* @param headers optionally request headers
261
+ * @param config optionally configuration
260
262
* @return response
261
263
* @throws IOException when error occured posting request and/or server failed
262
264
* to respond
263
265
*/
264
266
protected abstract ClickHouseHttpResponse post (String query , InputStream data , List <ClickHouseExternalTable > tables ,
265
- Map <String , String > headers ) throws IOException ;
267
+ String url , Map <String , String > headers , ClickHouseConfig config ) throws IOException ;
266
268
267
269
/**
268
270
* Checks whether the connection is reusable or not. This method will be called
@@ -287,36 +289,36 @@ protected boolean isReusable() {
287
289
public abstract boolean ping (int timeout );
288
290
289
291
public ClickHouseHttpResponse update (String query ) throws IOException {
290
- return post (query , null , null , null );
292
+ return post (query , null , null , null , null , null );
291
293
}
292
294
293
295
public ClickHouseHttpResponse update (String query , Map <String , String > headers ) throws IOException {
294
- return post (query , null , null , headers );
296
+ return post (query , null , null , null , headers , null );
295
297
}
296
298
297
299
public ClickHouseHttpResponse update (String query , InputStream data ) throws IOException {
298
- return post (query , data , null , null );
300
+ return post (query , data , null , null , null , null );
299
301
}
300
302
301
303
public ClickHouseHttpResponse update (String query , InputStream data , Map <String , String > headers )
302
304
throws IOException {
303
- return post (query , data , null , headers );
305
+ return post (query , data , null , null , headers , null );
304
306
}
305
307
306
308
public ClickHouseHttpResponse query (String query ) throws IOException {
307
- return post (query , null , null , null );
309
+ return post (query , null , null , null , null , null );
308
310
}
309
311
310
312
public ClickHouseHttpResponse query (String query , Map <String , String > headers ) throws IOException {
311
- return post (query , null , null , headers );
313
+ return post (query , null , null , null , headers , null );
312
314
}
313
315
314
316
public ClickHouseHttpResponse query (String query , List <ClickHouseExternalTable > tables ) throws IOException {
315
- return post (query , null , tables , null );
317
+ return post (query , null , tables , null , null , null );
316
318
}
317
319
318
320
public ClickHouseHttpResponse query (String query , List <ClickHouseExternalTable > tables , Map <String , String > headers )
319
321
throws IOException {
320
- return post (query , null , tables , headers );
322
+ return post (query , null , tables , null , headers , null );
321
323
}
322
324
}
0 commit comments