Skip to content

Commit 855fab7

Browse files
authored
Merge pull request #338 from Myllyenko/replacing_synchronized_with_reentrant_locks
Replacing synchronized blocks with ReentrantLocks in topic-related part of SDK
2 parents c513c4d + f887345 commit 855fab7

File tree

6 files changed

+157
-67
lines changed

6 files changed

+157
-67
lines changed

topic/src/main/java/tech/ydb/topic/impl/SessionBase.java

Lines changed: 56 additions & 37 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
import java.util.Objects;
44
import java.util.concurrent.CompletableFuture;
55
import java.util.concurrent.atomic.AtomicBoolean;
6+
import java.util.concurrent.locks.ReentrantLock;
67

78
import org.slf4j.Logger;
89

@@ -19,6 +20,7 @@ public abstract class SessionBase<R, W> implements Session {
1920

2021
protected final GrpcReadWriteStream<R, W> streamConnection;
2122
protected final AtomicBoolean isWorking = new AtomicBoolean(true);
23+
private final ReentrantLock lock = new ReentrantLock();
2224
private String token;
2325

2426
public SessionBase(GrpcReadWriteStream<R, W> streamConnection) {
@@ -32,57 +34,74 @@ public SessionBase(GrpcReadWriteStream<R, W> streamConnection) {
3234

3335
protected abstract void onStop();
3436

35-
protected synchronized CompletableFuture<Status> start(GrpcReadStream.Observer<R> streamObserver) {
36-
getLogger().info("Session start");
37-
return streamConnection.start(message -> {
38-
if (getLogger().isTraceEnabled()) {
39-
getLogger().trace("Message received:\n{}", message);
40-
} else {
41-
getLogger().debug("Message received");
42-
}
37+
protected CompletableFuture<Status> start(GrpcReadStream.Observer<R> streamObserver) {
38+
lock.lock();
39+
40+
try {
41+
getLogger().info("Session start");
42+
return streamConnection.start(message -> {
43+
if (getLogger().isTraceEnabled()) {
44+
getLogger().trace("Message received:\n{}", message);
45+
} else {
46+
getLogger().debug("Message received");
47+
}
48+
49+
if (isWorking.get()) {
50+
streamObserver.onNext(message);
51+
}
52+
});
53+
} finally {
54+
lock.unlock();
55+
}
56+
}
57+
58+
public void send(W request) {
59+
lock.lock();
4360

44-
if (isWorking.get()) {
45-
streamObserver.onNext(message);
61+
try {
62+
if (!isWorking.get()) {
63+
if (getLogger().isTraceEnabled()) {
64+
getLogger().trace("Session is already closed. This message is NOT sent:\n{}", request);
65+
}
66+
return;
67+
}
68+
String currentToken = streamConnection.authToken();
69+
if (!Objects.equals(token, currentToken)) {
70+
token = currentToken;
71+
getLogger().info("Sending new token");
72+
sendUpdateTokenRequest(token);
4673
}
47-
});
48-
}
4974

50-
public synchronized void send(W request) {
51-
if (!isWorking.get()) {
5275
if (getLogger().isTraceEnabled()) {
53-
getLogger().trace("Session is already closed. This message is NOT sent:\n{}", request);
76+
getLogger().trace("Sending request:\n{}", request);
77+
} else {
78+
getLogger().debug("Sending request");
5479
}
55-
return;
80+
streamConnection.sendNext(request);
81+
} finally {
82+
lock.unlock();
5683
}
57-
String currentToken = streamConnection.authToken();
58-
if (!Objects.equals(token, currentToken)) {
59-
token = currentToken;
60-
getLogger().info("Sending new token");
61-
sendUpdateTokenRequest(token);
62-
}
63-
64-
if (getLogger().isTraceEnabled()) {
65-
getLogger().trace("Sending request:\n{}", request);
66-
} else {
67-
getLogger().debug("Sending request");
68-
}
69-
streamConnection.sendNext(request);
7084
}
7185

7286
private boolean stop() {
7387
getLogger().info("Session stop");
7488
return isWorking.compareAndSet(true, false);
7589
}
7690

77-
7891
@Override
79-
public synchronized boolean shutdown() {
80-
getLogger().info("Session shutdown");
81-
if (stop()) {
82-
onStop();
83-
streamConnection.close();
84-
return true;
92+
public boolean shutdown() {
93+
lock.lock();
94+
95+
try {
96+
getLogger().info("Session shutdown");
97+
if (stop()) {
98+
onStop();
99+
streamConnection.close();
100+
return true;
101+
}
102+
return false;
103+
} finally {
104+
lock.unlock();
85105
}
86-
return false;
87106
}
88107
}

topic/src/main/java/tech/ydb/topic/read/impl/DeferredCommitterImpl.java

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
import java.util.List;
44
import java.util.Map;
55
import java.util.concurrent.ConcurrentHashMap;
6+
import java.util.concurrent.locks.ReentrantLock;
67

78
import org.slf4j.Logger;
89
import org.slf4j.LoggerFactory;
@@ -24,15 +25,20 @@ public class DeferredCommitterImpl implements DeferredCommitter {
2425
private static class PartitionRanges {
2526
private final PartitionSessionImpl partitionSession;
2627
private final DisjointOffsetRangeSet ranges = new DisjointOffsetRangeSet();
28+
private final ReentrantLock rangesLock = new ReentrantLock();
2729

2830
private PartitionRanges(PartitionSessionImpl partitionSession) {
2931
this.partitionSession = partitionSession;
3032
}
3133

3234
private void add(OffsetsRange offsetRange) {
3335
try {
34-
synchronized (ranges) {
36+
rangesLock.lock();
37+
38+
try {
3539
ranges.add(offsetRange);
40+
} finally {
41+
rangesLock.unlock();
3642
}
3743
} catch (RuntimeException exception) {
3844
String errorMessage = "Error adding new offset range to DeferredCommitter for partition session " +
@@ -45,8 +51,11 @@ private void add(OffsetsRange offsetRange) {
4551

4652
private void commit() {
4753
List<OffsetsRange> rangesToCommit;
48-
synchronized (ranges) {
54+
rangesLock.lock();
55+
try {
4956
rangesToCommit = ranges.getRangesAndClear();
57+
} finally {
58+
rangesLock.unlock();
5059
}
5160
partitionSession.commitOffsetRanges(rangesToCommit);
5261
}

topic/src/main/java/tech/ydb/topic/read/impl/PartitionSessionImpl.java

Lines changed: 32 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@
1212
import java.util.concurrent.ConcurrentSkipListMap;
1313
import java.util.concurrent.Executor;
1414
import java.util.concurrent.atomic.AtomicBoolean;
15+
import java.util.concurrent.locks.ReentrantLock;
1516
import java.util.function.Consumer;
1617
import java.util.function.Function;
1718
import java.util.stream.Collectors;
@@ -44,11 +45,13 @@ public class PartitionSessionImpl {
4445
private final AtomicBoolean isWorking = new AtomicBoolean(true);
4546

4647
private final Queue<Batch> decodingBatches = new LinkedList<>();
48+
private final ReentrantLock decodingBatchesLock = new ReentrantLock();
4749
private final Queue<Batch> readingQueue = new ConcurrentLinkedQueue<>();
4850
private final Function<DataReceivedEvent, CompletableFuture<Void>> dataEventCallback;
4951
private final AtomicBoolean isReadingNow = new AtomicBoolean();
5052
private final Consumer<List<OffsetsRange>> commitFunction;
5153
private final NavigableMap<Long, CompletableFuture<Void>> commitFutures = new ConcurrentSkipListMap<>();
54+
private final ReentrantLock commitFuturesLock = new ReentrantLock();
5255
// Offset of the last read message + 1
5356
private long lastReadOffset;
5457
private long lastCommittedOffset;
@@ -149,14 +152,21 @@ public CompletableFuture<Void> addBatches(List<YdbTopic.StreamReadMessage.ReadRe
149152
);
150153
});
151154
batchFutures.add(newBatch.getReadFuture());
152-
synchronized (decodingBatches) {
155+
156+
decodingBatchesLock.lock();
157+
158+
try {
153159
decodingBatches.add(newBatch);
160+
} finally {
161+
decodingBatchesLock.unlock();
154162
}
155163

156164
CompletableFuture.runAsync(() -> decode(newBatch), decompressionExecutor)
157165
.thenRun(() -> {
158166
boolean haveNewBatchesReady = false;
159-
synchronized (decodingBatches) {
167+
decodingBatchesLock.lock();
168+
169+
try {
160170
// Taking all encoded messages to sending queue
161171
while (true) {
162172
Batch decodingBatch = decodingBatches.peek();
@@ -176,7 +186,10 @@ public CompletableFuture<Void> addBatches(List<YdbTopic.StreamReadMessage.ReadRe
176186
break;
177187
}
178188
}
189+
} finally {
190+
decodingBatchesLock.unlock();
179191
}
192+
180193
if (haveNewBatchesReady) {
181194
sendDataToReadersIfNeeded();
182195
}
@@ -185,10 +198,12 @@ public CompletableFuture<Void> addBatches(List<YdbTopic.StreamReadMessage.ReadRe
185198
return CompletableFuture.allOf(batchFutures.toArray(new CompletableFuture<?>[0]));
186199
}
187200

188-
// Сommit single offset range with result future
201+
// Commit single offset range with result future
189202
public CompletableFuture<Void> commitOffsetRange(OffsetsRange rangeToCommit) {
190203
CompletableFuture<Void> resultFuture = new CompletableFuture<>();
191-
synchronized (commitFutures) {
204+
commitFuturesLock.lock();
205+
206+
try {
192207
if (isWorking.get()) {
193208
if (logger.isDebugEnabled()) {
194209
logger.debug("[{}] Offset range [{}, {}) is requested to be committed for partition session {} " +
@@ -205,6 +220,8 @@ public CompletableFuture<Void> commitOffsetRange(OffsetsRange rangeToCommit) {
205220
partitionId + ") for " + path + " is already closed"));
206221
return resultFuture;
207222
}
223+
} finally {
224+
commitFuturesLock.unlock();
208225
}
209226
List<OffsetsRange> rangeWrapper = new ArrayList<>(1);
210227
rangeWrapper.add(rangeToCommit);
@@ -334,16 +351,25 @@ private void sendDataToReadersIfNeeded() {
334351
}
335352

336353
public void shutdown() {
337-
synchronized (commitFutures) {
354+
commitFuturesLock.lock();
355+
356+
try {
338357
isWorking.set(false);
339358
logger.info("[{}] Partition session {} (partition {}) is shutting down. Failing {} commit futures...", path,
340359
id, partitionId, commitFutures.size());
341360
commitFutures.values().forEach(f -> f.completeExceptionally(new RuntimeException("Partition session " + id +
342361
" (partition " + partitionId + ") for " + path + " is closed")));
362+
} finally {
363+
commitFuturesLock.unlock();
343364
}
344-
synchronized (decodingBatches) {
365+
366+
decodingBatchesLock.lock();
367+
368+
try {
345369
decodingBatches.forEach(Batch::complete);
346370
readingQueue.forEach(Batch::complete);
371+
} finally {
372+
decodingBatchesLock.unlock();
347373
}
348374
}
349375

topic/src/main/java/tech/ydb/topic/read/impl/SyncReaderImpl.java

Lines changed: 21 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@
88
import java.util.Queue;
99
import java.util.concurrent.CompletableFuture;
1010
import java.util.concurrent.TimeUnit;
11+
import java.util.concurrent.locks.Condition;
12+
import java.util.concurrent.locks.ReentrantLock;
1113
import java.util.function.Consumer;
1214

1315
import javax.annotation.Nullable;
@@ -34,6 +36,8 @@ public class SyncReaderImpl extends ReaderImpl implements SyncReader {
3436
private static final Logger logger = LoggerFactory.getLogger(SyncReaderImpl.class);
3537
private static final int POLL_INTERVAL_SECONDS = 5;
3638
private final Queue<MessageBatchWrapper> batchesInQueue = new LinkedList<>();
39+
private final ReentrantLock queueLock = new ReentrantLock();
40+
private final Condition queueIsNotEmptyCondition = queueLock.newCondition();
3741
private int currentMessageIndex = 0;
3842

3943
public SyncReaderImpl(TopicRpc topicRpc, ReaderSettings settings) {
@@ -66,22 +70,21 @@ public Message receiveInternal(ReceiveSettings receiveSettings, long timeout, Ti
6670
if (isStopped.get()) {
6771
throw new RuntimeException("Reader was stopped");
6872
}
69-
synchronized (batchesInQueue) {
73+
74+
queueLock.lock();
75+
76+
try {
7077
if (batchesInQueue.isEmpty()) {
7178
long millisToWait = TimeUnit.MILLISECONDS.convert(timeout, unit);
7279
Instant deadline = Instant.now().plusMillis(millisToWait);
73-
while (true) {
74-
if (!batchesInQueue.isEmpty()) {
75-
break;
76-
}
77-
Instant now = Instant.now();
78-
if (now.isAfter(deadline)) {
80+
while (batchesInQueue.isEmpty()) {
81+
millisToWait = Duration.between(Instant.now(), deadline).toMillis();
82+
if (millisToWait <= 0) {
7983
break;
8084
}
81-
// Using Math.max to prevent rounding duration to 0 which would lead to infinite wait
82-
millisToWait = Math.max(1, Duration.between(now, deadline).toMillis());
85+
8386
logger.trace("No messages in queue. Waiting for {} ms...", millisToWait);
84-
batchesInQueue.wait(millisToWait);
87+
queueIsNotEmptyCondition.await(millisToWait, TimeUnit.MILLISECONDS);
8588
}
8689

8790
if (batchesInQueue.isEmpty()) {
@@ -112,6 +115,8 @@ public Message receiveInternal(ReceiveSettings receiveSettings, long timeout, Ti
112115
}
113116
}
114117
return result;
118+
} finally {
119+
queueLock.unlock();
115120
}
116121
}
117122

@@ -143,10 +148,14 @@ protected CompletableFuture<Void> handleDataReceivedEvent(DataReceivedEvent even
143148
return resultFuture;
144149
}
145150

146-
synchronized (batchesInQueue) {
151+
queueLock.lock();
152+
153+
try {
147154
logger.debug("Putting a message batch into queue and notifying in case receive method is waiting");
148155
batchesInQueue.add(new MessageBatchWrapper(event.getMessages(), resultFuture));
149-
batchesInQueue.notify();
156+
queueIsNotEmptyCondition.signal();
157+
} finally {
158+
queueLock.unlock();
150159
}
151160
return resultFuture;
152161
}

0 commit comments

Comments
 (0)