24
24
import java .util .*;
25
25
import java .util .concurrent .ConcurrentHashMap ;
26
26
import java .util .concurrent .ConcurrentLinkedQueue ;
27
+ import java .util .concurrent .ConcurrentMap ;
28
+ import java .util .concurrent .ConcurrentNavigableMap ;
29
+ import java .util .concurrent .ConcurrentSkipListMap ;
27
30
import java .util .concurrent .atomic .AtomicLong ;
28
31
29
32
import org .slf4j .Logger ;
30
33
import org .slf4j .LoggerFactory ;
31
34
32
35
final class KafkaRecordTracker {
33
36
private static final Logger log = LoggerFactory .getLogger (SplunkSinkTask .class );
34
- private ConcurrentHashMap <TopicPartition , TreeMap <Long , EventBatch >> all ; // TopicPartition + Long offset represents the SinkRecord
37
+ private ConcurrentMap <TopicPartition , ConcurrentNavigableMap <Long , EventBatch >> all ; // TopicPartition + Long offset represents the SinkRecord
35
38
private AtomicLong total ;
36
39
private ConcurrentLinkedQueue <EventBatch > failed ;
37
40
private volatile Map <TopicPartition , OffsetAndMetadata > offsets ;
@@ -57,7 +60,7 @@ public void removeAckedEventBatch(final EventBatch batch) {
57
60
final SinkRecord record = (SinkRecord ) event .getTied ();
58
61
TopicPartition tp = new TopicPartition (record .topic (), record .kafkaPartition ());
59
62
//log.debug("Processing topic {} partition {}", record.topic(), record.kafkaPartition());
60
- TreeMap <Long , EventBatch > tpRecords = all .get (tp );
63
+ ConcurrentNavigableMap <Long , EventBatch > tpRecords = all .get (tp );
61
64
if (tpRecords == null ) {
62
65
log .error ("KafkaRecordTracker removing a batch in an unknown partition {} {} {}" , record .topic (), record .kafkaPartition (), record .kafkaOffset ());
63
66
return ;
@@ -76,11 +79,7 @@ public void removeAckedEventBatch(final EventBatch batch) {
76
79
}
77
80
}
78
81
if (offset >= 0 ) {
79
- if (offsets .containsKey (tp )) {
80
- offsets .replace (tp , new OffsetAndMetadata (offset + 1 ));
81
- } else {
82
- offsets .put (tp , new OffsetAndMetadata (offset + 1 ));
83
- }
82
+ offsets .put (tp , new OffsetAndMetadata (offset + 1 ));
84
83
}
85
84
}
86
85
}
@@ -98,16 +97,8 @@ public void addEventBatch(final EventBatch batch) {
98
97
if (event .getTied () instanceof SinkRecord ) {
99
98
final SinkRecord record = (SinkRecord ) event .getTied ();
100
99
TopicPartition tp = new TopicPartition (record .topic (), record .kafkaPartition ());
101
- TreeMap <Long , EventBatch > tpRecords = all .get (tp );
102
- if (tpRecords == null ) {
103
- tpRecords = new TreeMap <>();
104
- all .put (tp , tpRecords );
105
- }
106
-
107
- if (!tpRecords .containsKey (record .kafkaOffset ())) {
108
- tpRecords .put (record .kafkaOffset (), batch );
109
- total .incrementAndGet ();
110
- }
100
+ ConcurrentNavigableMap <Long , EventBatch > tpRecords = all .computeIfAbsent (tp , k -> new ConcurrentSkipListMap <>());
101
+ tpRecords .computeIfAbsent (record .kafkaOffset (), k -> { total .incrementAndGet (); return batch ; });
111
102
}
112
103
}
113
104
}
0 commit comments