Skip to content
This repository was archived by the owner on Jan 24, 2024. It is now read-only.

Commit 6653115

Browse files
[fix] Make ProducerIdManagerImpl thread safe (#1603)
### Motivation The `ProducerIdManagerImpl` updates `currentProducerIdBlock` and `nextProducerId` from callbacks that are likely run on separate threads. When many producers connect, that could lead to data races that might lead to incorrect incrementing for producer ids. This PR seeks to ensure that getting the next `currentProducerIdBlock` is thread safe and does not get run concurrently, and to make sure that updates to `nextProducerId` are always thread safe. My model relies on synchronizing on the object, which is already partially implemented with synchronized methods. ### Modifications * Update the callback in `generateProducerId` to ensure that `nextProducerId` is updated safely and `currentProducerIdBlock` is read safely * Update `getNewProducerIdBlock` so that the `currentProducerIdBlock` is only updated after a successful write to the metadata store. * Introduce `newProducerIdBlockFuture` to prevent duplicate attempts to update the metadata store's state for `currentProducerIdBlock`. Duplicate attempts would fail, so this change should prevent races that will result in failure for all but one future. * Update the `nextProducerId` from within the `getNewProducerIdBlock` method to simplify the callback logic in the `generateProducerId` method.
1 parent 50eb5fd commit 6653115

File tree

2 files changed

+239
-48
lines changed

2 files changed

+239
-48
lines changed

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/coordinator/transaction/ProducerIdManagerImpl.java

+77-48
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,8 @@ public class ProducerIdManagerImpl implements ProducerIdManager {
5454
private ProducerIdBlock currentProducerIdBlock;
5555
private Long nextProducerId = -1L;
5656

57+
private CompletableFuture<Void> newProducerIdBlockFuture;
58+
5759
public ProducerIdManagerImpl(int brokerId, MetadataStoreExtended metadataStore) {
5860
this.brokerId = brokerId;
5961
this.metadataStore = metadataStore;
@@ -78,70 +80,88 @@ public static ProducerIdBlock parseProducerIdBlockData(byte[] bytes) throws IOEx
7880
.build();
7981
}
8082

81-
public CompletableFuture<Void> getNewProducerIdBlock() {
82-
CompletableFuture<Void> future = new CompletableFuture<>();
83+
public synchronized CompletableFuture<Void> getNewProducerIdBlock() {
84+
if (newProducerIdBlockFuture != null && !newProducerIdBlockFuture.isDone()) {
85+
// In this case, the class is already getting the new producer id block.
86+
// Returning this future ensures that callbacks work correctly
87+
return newProducerIdBlockFuture;
88+
}
89+
newProducerIdBlockFuture = new CompletableFuture<>();
8390
getCurrentDataAndVersion().thenAccept(currentDataAndVersionOpt -> {
84-
if (currentDataAndVersionOpt.isPresent() && currentDataAndVersionOpt.get().getData() != null) {
85-
DataAndVersion dataAndVersion = currentDataAndVersionOpt.get();
86-
try {
87-
ProducerIdBlock currProducerIdBlock =
88-
ProducerIdManagerImpl.parseProducerIdBlockData(dataAndVersion.getData());
89-
if (currProducerIdBlock.blockEndId > Long.MAX_VALUE - ProducerIdManagerImpl.PID_BLOCK_SIZE) {
90-
// We have exhausted all producerIds (wow!), treat it as a fatal error
91-
log.error("Exhausted all producerIds as the next block's end producerId is will "
92-
+ "has exceeded long type limit (current block end producerId is {})",
93-
currProducerIdBlock.blockEndId);
94-
future.completeExceptionally(new KafkaException("Have exhausted all producerIds."));
91+
synchronized (this) {
92+
final ProducerIdBlock nextProducerIdBlock;
93+
if (currentDataAndVersionOpt.isPresent() && currentDataAndVersionOpt.get().getData() != null) {
94+
DataAndVersion dataAndVersion = currentDataAndVersionOpt.get();
95+
try {
96+
ProducerIdBlock currProducerIdBlock =
97+
ProducerIdManagerImpl.parseProducerIdBlockData(dataAndVersion.getData());
98+
if (currProducerIdBlock.blockEndId > Long.MAX_VALUE - ProducerIdManagerImpl.PID_BLOCK_SIZE) {
99+
// We have exhausted all producerIds (wow!), treat it as a fatal error
100+
log.error("Exhausted all producerIds as the next block's end producerId is will "
101+
+ "has exceeded long type limit (current block end producerId is {})",
102+
currProducerIdBlock.blockEndId);
103+
newProducerIdBlockFuture
104+
.completeExceptionally(new KafkaException("Have exhausted all producerIds."));
105+
return;
106+
}
107+
nextProducerIdBlock = ProducerIdBlock
108+
.builder()
109+
.brokerId(brokerId)
110+
.blockStartId(currProducerIdBlock.blockEndId + 1L)
111+
.blockEndId(currProducerIdBlock.blockEndId + ProducerIdManagerImpl.PID_BLOCK_SIZE)
112+
.build();
113+
} catch (IOException e) {
114+
newProducerIdBlockFuture.completeExceptionally(new KafkaException("Get producerId failed.", e));
95115
return;
96116
}
97-
currentProducerIdBlock = ProducerIdBlock
117+
} else {
118+
if (log.isDebugEnabled()) {
119+
log.debug("There is no producerId block yet, creating the first block");
120+
}
121+
nextProducerIdBlock = ProducerIdBlock
98122
.builder()
99123
.brokerId(brokerId)
100-
.blockStartId(currProducerIdBlock.blockEndId + 1L)
101-
.blockEndId(currProducerIdBlock.blockEndId + ProducerIdManagerImpl.PID_BLOCK_SIZE)
124+
.blockStartId(0L)
125+
.blockEndId(ProducerIdManagerImpl.PID_BLOCK_SIZE - 1)
102126
.build();
103-
} catch (IOException e) {
104-
future.completeExceptionally(new KafkaException("Get producerId failed.", e));
105-
return;
106127
}
107-
} else {
108-
if (log.isDebugEnabled()) {
109-
log.debug("There is no producerId block yet, creating the first block");
128+
try {
129+
byte[] newProducerIdBlockData = ProducerIdManagerImpl
130+
.generateProducerIdBlockJson(nextProducerIdBlock);
131+
conditionalUpdateData(newProducerIdBlockData,
132+
currentDataAndVersionOpt.orElse(DataAndVersion.DEFAULT_VERSION).getVersion())
133+
.thenAccept(version -> {
134+
synchronized (this) {
135+
currentProducerIdBlock = nextProducerIdBlock;
136+
nextProducerId = nextProducerIdBlock.blockStartId;
137+
newProducerIdBlockFuture.complete(null);
138+
}
139+
}).exceptionally(ex -> {
140+
synchronized (this) {
141+
newProducerIdBlockFuture.completeExceptionally(ex);
142+
}
143+
return null;
144+
});
145+
} catch (JsonProcessingException e) {
146+
newProducerIdBlockFuture.completeExceptionally(e);
110147
}
111-
currentProducerIdBlock = ProducerIdBlock
112-
.builder()
113-
.brokerId(brokerId)
114-
.blockStartId(0L)
115-
.blockEndId(ProducerIdManagerImpl.PID_BLOCK_SIZE - 1)
116-
.build();
148+
}}).exceptionally(ex -> {
149+
synchronized (this) {
150+
newProducerIdBlockFuture.completeExceptionally(ex);
117151
}
118-
try {
119-
byte[] newProducerIdBlockData = ProducerIdManagerImpl
120-
.generateProducerIdBlockJson(currentProducerIdBlock);
121-
conditionalUpdateData(newProducerIdBlockData,
122-
currentDataAndVersionOpt.orElse(DataAndVersion.DEFAULT_VERSION).getVersion())
123-
.thenAccept(version -> {
124-
future.complete(null);
125-
}).exceptionally(ex -> {
126-
future.completeExceptionally(ex);
127-
return null;
128-
});
129-
} catch (JsonProcessingException e) {
130-
future.completeExceptionally(e);
131-
}
132-
}).exceptionally(ex -> {
133-
future.completeExceptionally(ex);
134152
return null;
135153
});
136-
return future;
154+
return newProducerIdBlockFuture;
137155
}
138156

139157
@Override
140158
public CompletableFuture<Void> initialize() {
141159
CompletableFuture<Void> future = new CompletableFuture<>();
142160
getNewProducerIdBlock()
143161
.thenAccept(__ -> {
144-
nextProducerId = currentProducerIdBlock.blockStartId;
162+
synchronized (this) {
163+
nextProducerId = currentProducerIdBlock.blockStartId;
164+
}
145165
future.complete(null);
146166
}).exceptionally(throwable -> {
147167
future.completeExceptionally(throwable);
@@ -156,8 +176,17 @@ public synchronized CompletableFuture<Long> generateProducerId() {
156176
// grab a new block of producerIds if this block has been exhausted
157177
if (nextProducerId > currentProducerIdBlock.blockEndId) {
158178
getNewProducerIdBlock().thenAccept(__ -> {
159-
nextProducerId = currentProducerIdBlock.blockStartId + 1;
160-
nextProducerIdFuture.complete(nextProducerId - 1);
179+
synchronized (this) {
180+
if (nextProducerId > currentProducerIdBlock.blockEndId) {
181+
// This can only happen if more than blockSize producers attempt to connect
182+
// while the getNewProducerIdBlock() is processing
183+
Exception ex = new IllegalStateException("New ProducerIdBlock exhausted. Try again.");
184+
nextProducerIdFuture.completeExceptionally(ex);
185+
} else {
186+
nextProducerId += 1;
187+
nextProducerIdFuture.complete(nextProducerId - 1);
188+
}
189+
}
161190
}).exceptionally(ex -> {
162191
nextProducerIdFuture.completeExceptionally(ex);
163192
return null;
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,162 @@
1+
/**
2+
* Licensed under the Apache License, Version 2.0 (the "License");
3+
* you may not use this file except in compliance with the License.
4+
* You may obtain a copy of the License at
5+
*
6+
* http://www.apache.org/licenses/LICENSE-2.0
7+
*
8+
* Unless required by applicable law or agreed to in writing, software
9+
* distributed under the License is distributed on an "AS IS" BASIS,
10+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
11+
* See the License for the specific language governing permissions and
12+
* limitations under the License.
13+
*/
14+
package io.streamnative.pulsar.handlers.kop.coordinator.transaction;
15+
16+
import static org.mockito.ArgumentMatchers.any;
17+
import static org.mockito.ArgumentMatchers.anyString;
18+
import static org.mockito.Mockito.mock;
19+
import static org.mockito.Mockito.when;
20+
21+
import java.util.ArrayList;
22+
import java.util.HashSet;
23+
import java.util.List;
24+
import java.util.Optional;
25+
import java.util.concurrent.CompletableFuture;
26+
import org.apache.pulsar.metadata.api.GetResult;
27+
import org.apache.pulsar.metadata.api.MetadataStoreConfig;
28+
import org.apache.pulsar.metadata.api.Stat;
29+
import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
30+
import org.apache.pulsar.metadata.impl.LocalMemoryMetadataStore;
31+
import org.testng.Assert;
32+
import org.testng.annotations.Test;
33+
34+
public class ProducerIdManagerImplTest {
35+
36+
@Test
37+
public void verifyThreadSafetyForTwoConcurrentNewProducerIdBlockCalls() throws Exception {
38+
// Initialize a fake metadata store such that the futures are not completed. This will allow
39+
// for low level control during this test.
40+
CompletableFuture<Optional<GetResult>> getFuture = new CompletableFuture<>();
41+
CompletableFuture<Stat> completedPutFuture = new CompletableFuture<>();
42+
// The value is not used, so mock with all "zero" values
43+
completedPutFuture.complete(new Stat("", 0, 0, 0, false, false));
44+
45+
MetadataStoreExtended mockedMetadataStore = mock(MetadataStoreExtended.class);
46+
when(mockedMetadataStore.get(anyString())).thenReturn(getFuture);
47+
when(mockedMetadataStore.put(anyString(), any(), any())).thenReturn(completedPutFuture);
48+
49+
ProducerIdManagerImpl producerIdManager = new ProducerIdManagerImpl(1, mockedMetadataStore);
50+
// Trigger two calls to increase the producer id block.
51+
CompletableFuture<Void> firstNewBlock = producerIdManager.getNewProducerIdBlock();
52+
CompletableFuture<Void> secondNewBlock = producerIdManager.getNewProducerIdBlock();
53+
54+
Assert.assertFalse(firstNewBlock.isDone());
55+
Assert.assertFalse(secondNewBlock.isDone());
56+
57+
// Relies on the fact that completing the future also triggers the callbacks to run in same thread
58+
getFuture.complete(Optional.empty());
59+
60+
// Ensure that both calls completed
61+
Assert.assertTrue(firstNewBlock.isDone());
62+
Assert.assertTrue(secondNewBlock.isDone());
63+
Assert.assertFalse(firstNewBlock.isCompletedExceptionally());
64+
Assert.assertFalse(secondNewBlock.isCompletedExceptionally());
65+
66+
// Ensure that the next producer id is the first value
67+
Assert.assertEquals(producerIdManager.generateProducerId().get().intValue(), 0, "The first id should be 0.");
68+
}
69+
70+
@Test
71+
public void verifyProducerIdManagerForManyBrokersAndManyNewProducers() throws Exception {
72+
int expectedNumIds = 1000000;
73+
int numBrokers = 10;
74+
LocalMemoryMetadataStore metadataStore =
75+
new LocalMemoryMetadataStore("memory:localhost", MetadataStoreConfig.builder().build());
76+
List<ProducerIdManagerImpl> producerIdManagers = new ArrayList<>(numBrokers);
77+
for (int i = 0; i < numBrokers; i++) {
78+
ProducerIdManagerImpl producerIdManager = new ProducerIdManagerImpl(i, metadataStore);
79+
producerIdManagers.add(producerIdManager);
80+
producerIdManager.initialize();
81+
}
82+
83+
List<CompletableFuture<Long>> futureIds = new ArrayList<>(expectedNumIds);
84+
85+
for (int i = 0; i < expectedNumIds; i++) {
86+
for (ProducerIdManagerImpl producerIdManager : producerIdManagers) {
87+
futureIds.add(producerIdManager.generateProducerId());
88+
}
89+
}
90+
91+
CompletableFuture.allOf(futureIds.toArray(new CompletableFuture[0])).get();
92+
93+
HashSet<Long> ids = new HashSet<>();
94+
for (CompletableFuture<Long> futureId : futureIds) {
95+
Assert.assertTrue(ids.add(futureId.get()), String.format("Expected %d to be a unique id", futureId.get()));
96+
}
97+
Assert.assertEquals(ids.size(), expectedNumIds * numBrokers);
98+
}
99+
100+
@Test
101+
public void tooManyConcurrentNewProducersShouldFail() throws Exception {
102+
long blockSize = ProducerIdManagerImpl.PID_BLOCK_SIZE;
103+
int brokerId = 1;
104+
// Initialize a fake metadata store such that the futures are not completed. This will allow
105+
// for low level control during this test.
106+
CompletableFuture<Optional<GetResult>> firstGetFuture = new CompletableFuture<>();
107+
CompletableFuture<Optional<GetResult>> secondGetFuture = new CompletableFuture<>();
108+
CompletableFuture<Stat> firstPutFuture = new CompletableFuture<>();
109+
// The value is not used, and we mock the get results, so the put is essentially ignored
110+
firstPutFuture.complete(new Stat("", 0, 0, 0, false, false));
111+
112+
MetadataStoreExtended mockedMetadataStore = mock(MetadataStoreExtended.class);
113+
when(mockedMetadataStore.get(anyString())).thenReturn(firstGetFuture).thenReturn(secondGetFuture);
114+
when(mockedMetadataStore.put(anyString(), any(), any())).thenReturn(firstPutFuture);
115+
116+
ProducerIdManagerImpl producerIdManager = new ProducerIdManagerImpl(brokerId, mockedMetadataStore);
117+
producerIdManager.initialize();
118+
// Relies on the fact that completing the future also triggers the callbacks to run
119+
firstGetFuture.complete(Optional.empty());
120+
List<CompletableFuture<Long>> futureIds = new ArrayList<>((int) blockSize + 1);
121+
122+
// Create one blockSize worth of producer ids
123+
for (int i = 0; i < blockSize; i++) {
124+
Assert.assertEquals(producerIdManager.generateProducerId().get().intValue(), i);
125+
}
126+
127+
// Now create callbacks for blockSize + 1 producer ids.
128+
for (int i = 0; i < blockSize + 1; i++) {
129+
futureIds.add(producerIdManager.generateProducerId());
130+
}
131+
132+
// Relies on the fact that completing the future also triggers the callbacks to run
133+
ProducerIdManagerImpl.ProducerIdBlock zeroBlock = ProducerIdManagerImpl.ProducerIdBlock
134+
.builder()
135+
.brokerId(brokerId)
136+
.blockStartId(0L)
137+
.blockEndId(ProducerIdManagerImpl.PID_BLOCK_SIZE - 1)
138+
.build();
139+
// This stat is not actually used
140+
Stat stat = new Stat("", 0, 0, 0, false, false);
141+
GetResult result = new GetResult(ProducerIdManagerImpl.generateProducerIdBlockJson(zeroBlock), stat);
142+
secondGetFuture.complete(Optional.of(result));
143+
144+
int countFailed = 0;
145+
HashSet<Long> set = new HashSet<>();
146+
for (CompletableFuture<Long> id : futureIds) {
147+
if (id.isDone()) {
148+
if (id.isCompletedExceptionally()) {
149+
countFailed++;
150+
} else {
151+
set.add(id.get());
152+
}
153+
} else {
154+
Assert.fail();
155+
}
156+
}
157+
158+
Assert.assertEquals(countFailed, 1, "Only one producer id should have failed");
159+
Assert.assertEquals(set.size(), blockSize, "Ensures all ids are unique and that no extra ids were created.");
160+
}
161+
162+
}

0 commit comments

Comments
 (0)