15
15
import java .util .concurrent .ExecutorService ;
16
16
import java .util .concurrent .Executors ;
17
17
import java .util .concurrent .ScheduledExecutorService ;
18
+ import java .util .concurrent .TimeUnit ;
19
+ import java .util .concurrent .TimeoutException ;
18
20
import java .util .concurrent .atomic .AtomicReference ;
19
21
20
22
import com .clickhouse .client .config .ClickHouseOption ;
@@ -80,6 +82,10 @@ public boolean ping(ClickHouseNode server, int timeout) {
80
82
static final class Agent implements ClickHouseClient {
81
83
private static final Logger log = LoggerFactory .getLogger (Agent .class );
82
84
85
+ private static final long INITIAL_REPEAT_DELAY = 100 ;
86
+ private static final long MAX_REPEAT_DELAY = 1000 ;
87
+ private static final long REPEAT_DELAY_BACKOFF = 100 ;
88
+
83
89
private final AtomicReference <ClickHouseClient > client ;
84
90
85
91
Agent (ClickHouseClient client , ClickHouseConfig config ) {
@@ -110,6 +116,7 @@ ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, ClickHouseExcept
110
116
ClickHouseNode current = sealedRequest .getServer ();
111
117
ClickHouseNodeManager manager = current .manager .get ();
112
118
if (manager == null ) {
119
+ log .debug ("Cancel failover for unmanaged node: %s" , current );
113
120
break ;
114
121
}
115
122
ClickHouseNode next = manager .suggestNode (current , exception );
@@ -118,8 +125,10 @@ ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, ClickHouseExcept
118
125
break ;
119
126
}
120
127
current .update (Status .FAULTY );
121
- next = sealedRequest .changeServer (current , next );
122
- if (next == current ) {
128
+ if (sealedRequest .isTransactional ()) {
129
+ log .debug ("Cancel failover for transactional context: %s" , sealedRequest .getTransaction ());
130
+ break ;
131
+ } else if ((next = sealedRequest .changeServer (current , next )) == current ) {
123
132
log .debug ("Cancel failover for no alternative of %s" , current );
124
133
break ;
125
134
}
@@ -162,6 +171,59 @@ ClickHouseResponse failover(ClickHouseRequest<?> sealedRequest, ClickHouseExcept
162
171
throw new CompletionException (exception );
163
172
}
164
173
174
+ /**
175
+ * Repeats sending same request until success, timed out or running into a
176
+ * different error.
177
+ *
178
+ * @param sealedRequest non-null sealed request
179
+ * @param exception non-null exception to start with
180
+ * @param timeout timeout in milliseconds, zero or negative numbers means
181
+ * no repeat
182
+ * @return non-null response
183
+ * @throws CompletionException when error occurred or timed out
184
+ */
185
+ ClickHouseResponse repeat (ClickHouseRequest <?> sealedRequest , ClickHouseException exception , long timeout ) {
186
+ if (timeout > 0 ) {
187
+ final int errorCode = exception .getErrorCode ();
188
+ final long startTime = System .currentTimeMillis ();
189
+
190
+ long delay = INITIAL_REPEAT_DELAY ;
191
+ long elapsed = 0L ;
192
+ int count = 1 ;
193
+ while (true ) {
194
+ log .info ("Repeating #%d (delay=%d, elapsed=%d, timeout=%d) due to: %s" , count ++, delay , elapsed ,
195
+ timeout , exception .getMessage ());
196
+ try {
197
+ return sendOnce (sealedRequest );
198
+ } catch (Exception exp ) {
199
+ exception = ClickHouseException .of (exp .getCause () != null ? exp .getCause () : exp ,
200
+ sealedRequest .getServer ());
201
+ }
202
+
203
+ elapsed = System .currentTimeMillis () - startTime ;
204
+ if (exception .getErrorCode () != errorCode || elapsed + delay >= timeout ) {
205
+ log .warn ("Stopped repeating(delay=%d, elapsed=%d, timeout=%d) for %s" , delay , elapsed ,
206
+ timeout , exception .getMessage ());
207
+ break ;
208
+ }
209
+
210
+ try {
211
+ Thread .sleep (delay );
212
+ elapsed += delay ;
213
+ } catch (InterruptedException e ) {
214
+ Thread .currentThread ().interrupt ();
215
+ break ;
216
+ }
217
+ if (delay >= MAX_REPEAT_DELAY ) {
218
+ delay = MAX_REPEAT_DELAY ;
219
+ } else {
220
+ delay += REPEAT_DELAY_BACKOFF ;
221
+ }
222
+ }
223
+ }
224
+ throw new CompletionException (exception );
225
+ }
226
+
165
227
ClickHouseResponse retry (ClickHouseRequest <?> sealedRequest , ClickHouseException exception , int times ) {
166
228
for (int i = 1 ; i <= times ; i ++) {
167
229
log .debug ("Retry %d of %d due to: %s" , i , times , exception .getMessage ());
@@ -186,18 +248,27 @@ ClickHouseResponse handle(ClickHouseRequest<?> sealedRequest, Throwable cause) {
186
248
cause = ((UncheckedIOException ) cause ).getCause ();
187
249
}
188
250
189
- log .debug ("Handling %s(failover=%d, retry=%d)" , cause , sealedRequest .getConfig ().getFailover (),
190
- sealedRequest .getConfig ().getRetry ());
251
+ ClickHouseConfig config = sealedRequest .getConfig ();
252
+ log .debug ("Handling %s(failover=%d, retry=%d)" , cause , config .getFailover (), config .getRetry ());
253
+ ClickHouseException ex = ClickHouseException .of (cause , sealedRequest .getServer ());
191
254
try {
255
+ if (config .isRepeatOnSessionLock ()
256
+ && ex .getErrorCode () == ClickHouseException .ERROR_SESSION_IS_LOCKED ) {
257
+ // connection timeout is usually a small number(defaults to 5000 ms), making it
258
+ // better default compare to socket timeout and max execution time etc.
259
+ return repeat (sealedRequest , ex , config .getSessionTimeout () <= 0 ? config .getConnectionTimeout ()
260
+ : TimeUnit .SECONDS .toMillis (config .getSessionTimeout ()));
261
+ }
262
+
192
263
int times = sealedRequest .getConfig ().getFailover ();
193
264
if (times > 0 ) {
194
- return failover (sealedRequest , ClickHouseException . of ( cause , sealedRequest . getServer ()) , times );
265
+ return failover (sealedRequest , ex , times );
195
266
}
196
267
197
268
// different from failover: 1) retry on the same node; 2) never retry on timeout
198
269
times = sealedRequest .getConfig ().getRetry ();
199
270
if (times > 0 ) {
200
- return retry (sealedRequest , ClickHouseException . of ( cause , sealedRequest . getServer ()) , times );
271
+ return retry (sealedRequest , ex , times );
201
272
}
202
273
203
274
throw new CompletionException (cause );
@@ -210,11 +281,12 @@ ClickHouseResponse handle(ClickHouseRequest<?> sealedRequest, Throwable cause) {
210
281
211
282
ClickHouseResponse sendOnce (ClickHouseRequest <?> sealedRequest ) {
212
283
try {
213
- return getClient ().execute (sealedRequest ).get ();
284
+ return getClient ().execute (sealedRequest ).get (sealedRequest .getConfig ().getSocketTimeout (),
285
+ TimeUnit .MILLISECONDS );
214
286
} catch (InterruptedException e ) {
215
287
Thread .currentThread ().interrupt ();
216
288
throw new CancellationException ("Execution was interrupted" );
217
- } catch (ExecutionException e ) {
289
+ } catch (ExecutionException | TimeoutException e ) {
218
290
throw new CompletionException (e .getCause ());
219
291
}
220
292
}
0 commit comments