29
29
import com .google .common .base .Joiner ;
30
30
import com .google .common .base .Optional ;
31
31
import com .google .common .base .Preconditions ;
32
- import com .google .common .base .Predicate ;
33
32
import com .google .common .base .Supplier ;
34
33
import com .google .common .base .Throwables ;
35
34
import com .google .common .collect .ImmutableList ;
40
39
import com .google .common .collect .Sets ;
41
40
import com .google .common .primitives .Ints ;
42
41
import com .metamx .common .ISE ;
43
- import com .metamx .common .RetryUtils ;
44
42
import com .metamx .common .guava .Sequence ;
45
43
import com .metamx .common .logger .Logger ;
46
44
import com .metamx .common .parsers .ParseException ;
97
95
import java .util .Properties ;
98
96
import java .util .Random ;
99
97
import java .util .Set ;
100
- import java .util .concurrent .Callable ;
101
98
import java .util .concurrent .ConcurrentHashMap ;
102
99
import java .util .concurrent .TimeUnit ;
103
100
import java .util .concurrent .locks .Condition ;
@@ -121,6 +118,7 @@ public enum Status
121
118
private static final String TYPE = "index_kafka" ;
122
119
private static final Random RANDOM = new Random ();
123
120
private static final long POLL_TIMEOUT = 100 ;
121
+ private static final long POLL_RETRY_MS = 30000 ;
124
122
private static final String METADATA_NEXT_PARTITIONS = "nextPartitions" ;
125
123
126
124
private final DataSchema dataSchema ;
@@ -161,6 +159,8 @@ public enum Status
161
159
private final Lock pauseLock = new ReentrantLock ();
162
160
private final Condition hasPaused = pauseLock .newCondition ();
163
161
private final Condition shouldResume = pauseLock .newCondition ();
162
+ private final Lock pollRetryLock = new ReentrantLock ();
163
+ private final Condition isAwaitingRetry = pollRetryLock .newCondition ();
164
164
private volatile boolean pauseRequested = false ;
165
165
private volatile long pauseMillis = 0 ;
166
166
@@ -342,6 +342,7 @@ public void run()
342
342
// Main loop.
343
343
// Could eventually support leader/follower mode (for keeping replicas more in sync)
344
344
boolean stillReading = !assignment .isEmpty ();
345
+ status = Status .READING ;
345
346
try {
346
347
while (stillReading ) {
347
348
if (possiblyPause (assignment )) {
@@ -363,30 +364,23 @@ public void run()
363
364
// The retrying business is because the KafkaConsumer throws OffsetOutOfRangeException if the seeked-to
364
365
// offset is not present in the topic-partition. This can happen if we're asking a task to read from data
365
366
// that has not been written yet (which is totally legitimate). So let's wait for it to show up.
366
- final ConsumerRecords <byte [], byte []> records = RetryUtils .retry (
367
- new Callable <ConsumerRecords <byte [], byte []>>()
368
- {
369
- @ Override
370
- public ConsumerRecords <byte [], byte []> call () throws Exception
371
- {
372
- try {
373
- return consumer .poll (POLL_TIMEOUT );
374
- }
375
- finally {
376
- status = Status .READING ;
377
- }
378
- }
379
- },
380
- new Predicate <Throwable >()
381
- {
382
- @ Override
383
- public boolean apply (Throwable input )
384
- {
385
- return input instanceof OffsetOutOfRangeException ;
386
- }
387
- },
388
- Integer .MAX_VALUE
389
- );
367
+ ConsumerRecords <byte [], byte []> records = ConsumerRecords .empty ();
368
+ try {
369
+ records = consumer .poll (POLL_TIMEOUT );
370
+ }
371
+ catch (OffsetOutOfRangeException e ) {
372
+ log .warn ("OffsetOutOfRangeException with message [%s], retrying in %dms" , e .getMessage (), POLL_RETRY_MS );
373
+ pollRetryLock .lockInterruptibly ();
374
+ try {
375
+ long nanos = TimeUnit .MILLISECONDS .toNanos (POLL_RETRY_MS );
376
+ while (nanos > 0L && !pauseRequested ) {
377
+ nanos = isAwaitingRetry .awaitNanos (nanos );
378
+ }
379
+ }
380
+ finally {
381
+ pollRetryLock .unlock ();
382
+ }
383
+ }
390
384
391
385
for (ConsumerRecord <byte [], byte []> record : records ) {
392
386
if (log .isTraceEnabled ()) {
@@ -689,6 +683,14 @@ public Response pause(@QueryParam("timeout") @DefaultValue("0") final long timeo
689
683
pauseMillis = timeout <= 0 ? PAUSE_FOREVER : timeout ;
690
684
pauseRequested = true ;
691
685
686
+ pollRetryLock .lockInterruptibly ();
687
+ try {
688
+ isAwaitingRetry .signalAll ();
689
+ }
690
+ finally {
691
+ pollRetryLock .unlock ();
692
+ }
693
+
692
694
if (isPaused ()) {
693
695
shouldResume .signalAll (); // kick the monitor so it re-awaits with the new pauseMillis
694
696
}
0 commit comments