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

Commit 1dda879

Browse files
gaoran10eolivelli
andauthored
[feature][transaction] Purge old aborted txn (#1879)
This PR needs producer state manager recovery. ### Motivation Purge old aborted txns. ### Modifications Add a timer task to purge useless old aborted data in the cache. Co-authored-by: Enrico Olivelli <[email protected]>
1 parent e69ad67 commit 1dda879

File tree

10 files changed

+410
-72
lines changed

10 files changed

+410
-72
lines changed

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaProtocolHandler.java

+17
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747
import java.util.concurrent.CompletableFuture;
4848
import java.util.concurrent.ConcurrentHashMap;
4949
import java.util.concurrent.ExecutionException;
50+
import java.util.concurrent.ScheduledFuture;
5051
import java.util.concurrent.TimeUnit;
5152
import java.util.concurrent.TimeoutException;
5253
import java.util.function.Function;
@@ -117,6 +118,8 @@ public class KafkaProtocolHandler implements ProtocolHandler, TenantContextManag
117118
private MigrationManager migrationManager;
118119
private ReplicaManager replicaManager;
119120

121+
private ScheduledFuture<?> txUpdatedPurgeAbortedTxOffsetsTimeHandle;
122+
120123
private final Map<String, GroupCoordinator> groupCoordinatorsByTenant = new ConcurrentHashMap<>();
121124
private final Map<String, TransactionCoordinator> transactionCoordinatorByTenant = new ConcurrentHashMap<>();
122125

@@ -309,6 +312,16 @@ private void invalidatePartitionLog(TopicName topicName) {
309312
schemaRegistryManager = new SchemaRegistryManager(kafkaConfig, brokerService.getPulsar(),
310313
brokerService.getAuthenticationService());
311314
migrationManager = new MigrationManager(kafkaConfig, brokerService.getPulsar());
315+
316+
if (kafkaConfig.isKafkaTransactionCoordinatorEnabled()
317+
&& kafkaConfig.getKafkaTxnPurgeAbortedTxnIntervalSeconds() > 0) {
318+
txUpdatedPurgeAbortedTxOffsetsTimeHandle = service.getPulsar().getExecutor().scheduleWithFixedDelay(() -> {
319+
getReplicaManager().updatePurgeAbortedTxnsOffsets();
320+
},
321+
kafkaConfig.getKafkaTxnPurgeAbortedTxnIntervalSeconds(),
322+
kafkaConfig.getKafkaTxnPurgeAbortedTxnIntervalSeconds(),
323+
TimeUnit.SECONDS);
324+
}
312325
}
313326

314327
private TransactionCoordinator createAndBootTransactionCoordinator(String tenant) {
@@ -522,6 +535,10 @@ public Map<InetSocketAddress, ChannelInitializer<SocketChannel>> newChannelIniti
522535

523536
@Override
524537
public void close() {
538+
if (txUpdatedPurgeAbortedTxOffsetsTimeHandle != null) {
539+
txUpdatedPurgeAbortedTxOffsetsTimeHandle.cancel(false);
540+
}
541+
525542
if (producePurgatory != null) {
526543
producePurgatory.shutdown();
527544
}

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaServiceConfiguration.java

+6
Original file line numberDiff line numberDiff line change
@@ -451,6 +451,12 @@ public class KafkaServiceConfiguration extends ServiceConfiguration {
451451
)
452452
private int kafkaTransactionRecoveryNumThreads = 8;
453453

454+
@FieldContext(
455+
category = CATEGORY_KOP_TRANSACTION,
456+
doc = "Interval for purging aborted transactions from memory (requires reads from storage)"
457+
)
458+
private int kafkaTxnPurgeAbortedTxnIntervalSeconds = 60 * 60;
459+
454460
@FieldContext(
455461
category = CATEGORY_KOP_TRANSACTION,
456462
doc = "The interval in milliseconds at which to rollback transactions that have timed out."

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/scala/Either.java

+2
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
import java.util.function.Consumer;
1717
import java.util.function.Function;
1818
import lombok.Getter;
19+
import lombok.ToString;
1920

2021
/**
2122
* A simple Java migration of <a href="https://www.scala-lang.org/api/2.13.6/scala/util/Either.html">Scala Either</a>.
@@ -45,6 +46,7 @@
4546
* @param <W> the type of the 2nd possible value (the right side)
4647
*/
4748
@Getter
49+
@ToString
4850
public class Either<V, W> {
4951

5052
private final V left;

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/storage/PartitionLog.java

+38-3
Original file line numberDiff line numberDiff line change
@@ -203,9 +203,12 @@ private CompletableFuture<Void> loadTopicProperties() {
203203
this.entryFormatter = buildEntryFormatter(topicProperties);
204204
this.kafkaTopicUUID = properties.get("kafkaTopicUUID");
205205
this.producerStateManager =
206-
new ProducerStateManager(fullPartitionName, kafkaTopicUUID,
206+
new ProducerStateManager(
207+
fullPartitionName,
208+
kafkaTopicUUID,
207209
producerStateManagerSnapshotBuffer,
208-
kafkaConfig.getKafkaTxnProducerStateTopicSnapshotIntervalSeconds());
210+
kafkaConfig.getKafkaTxnProducerStateTopicSnapshotIntervalSeconds(),
211+
kafkaConfig.getKafkaTxnPurgeAbortedTxnIntervalSeconds());
209212
});
210213
}
211214

@@ -1064,6 +1067,25 @@ private MemoryRecords trimInvalidBytes(MemoryRecords records, LogAppendInfo info
10641067
}
10651068
}
10661069

1070+
/**
1071+
* Remove all the AbortedTxn that are no more referred by existing data on the topic.
1072+
* @return
1073+
*/
1074+
public CompletableFuture<?> updatePurgeAbortedTxnsOffset() {
1075+
if (!kafkaConfig.isKafkaTransactionCoordinatorEnabled()) {
1076+
// no need to scan the topic, because transactions are disabled
1077+
return CompletableFuture.completedFuture(null);
1078+
}
1079+
if (!producerStateManager.hasSomeAbortedTransactions()) {
1080+
// nothing to do
1081+
return CompletableFuture.completedFuture(null);
1082+
}
1083+
return fetchOldestAvailableIndexFromTopic()
1084+
.thenAccept(offset ->
1085+
producerStateManager.updateAbortedTxnsPurgeOffset(offset));
1086+
1087+
}
1088+
10671089
public CompletableFuture<Long> fetchOldestAvailableIndexFromTopic() {
10681090
final CompletableFuture<Long> future = new CompletableFuture<>();
10691091

@@ -1130,6 +1152,19 @@ public CompletableFuture<?> takeProducerSnapshot() {
11301152
});
11311153
}
11321154

1155+
public CompletableFuture<Long> forcePurgeAbortTx() {
1156+
return initFuture.thenCompose((___) -> {
1157+
// purge can be taken only on the same thread that is used for writes
1158+
ManagedLedgerImpl ml = (ManagedLedgerImpl) getPersistentTopic().getManagedLedger();
1159+
ExecutorService executorService = ml.getScheduledExecutor().chooseThread(ml.getName());
1160+
1161+
return updatePurgeAbortedTxnsOffset()
1162+
.thenApplyAsync((____) -> {
1163+
return getProducerStateManager().executePurgeAbortedTx();
1164+
}, executorService);
1165+
});
1166+
}
1167+
11331168
public CompletableFuture<Long> recoverTxEntries(
11341169
long offset,
11351170
Executor executor) {
@@ -1206,7 +1241,6 @@ public CompletableFuture<Long> recoverTxEntries(
12061241
}));
12071242
}
12081243

1209-
12101244
private void readNextEntriesForRecovery(ManagedCursor cursor, AtomicLong cursorOffset,
12111245
KafkaTopicConsumerManager tcm,
12121246
AtomicLong entryCounter,
@@ -1310,6 +1344,7 @@ private void updateProducerStateManager(long lastOffset, AnalyzeResult analyzeRe
13101344

13111345
// do system clean up stuff in this thread
13121346
producerStateManager.maybeTakeSnapshot(recoveryExecutor);
1347+
producerStateManager.maybePurgeAbortedTx();
13131348
}
13141349

13151350
private void decodeEntriesForRecovery(final CompletableFuture<DecodeResult> future,

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/storage/PartitionLogManager.java

+5-7
Original file line numberDiff line numberDiff line change
@@ -107,17 +107,15 @@ public int size() {
107107
return logMap.size();
108108
}
109109

110-
public CompletableFuture<Void> takeProducerStateSnapshots() {
111-
List<CompletableFuture<Void>> handles = new ArrayList<>();
110+
public CompletableFuture<?> updatePurgeAbortedTxnsOffsets() {
111+
List<CompletableFuture<?>> handles = new ArrayList<>();
112112
logMap.values().forEach(log -> {
113113
if (log.isInitialised()) {
114-
handles.add(log
115-
.getProducerStateManager()
116-
.takeSnapshot(recoveryExecutor)
117-
.thenApply(___ -> null));
114+
handles.add(log.updatePurgeAbortedTxnsOffset());
118115
}
119116
});
120-
return FutureUtil.waitForAll(handles);
117+
return FutureUtil
118+
.waitForAll(handles);
121119
}
122120
}
123121

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/storage/ProducerStateManager.java

+67-7
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
*/
1414
package io.streamnative.pulsar.handlers.kop.storage;
1515

16+
import com.google.common.annotations.VisibleForTesting;
1617
import com.google.common.collect.Maps;
1718
import java.util.ArrayList;
1819
import java.util.HashMap;
@@ -22,6 +23,7 @@
2223
import java.util.TreeMap;
2324
import java.util.concurrent.CompletableFuture;
2425
import java.util.concurrent.Executor;
26+
import java.util.concurrent.atomic.AtomicLong;
2527
import lombok.Getter;
2628
import lombok.extern.slf4j.Slf4j;
2729
import org.apache.bookkeeper.util.SafeRunnable;
@@ -47,21 +49,27 @@ public class ProducerStateManager {
4749
private final ProducerStateManagerSnapshotBuffer producerStateManagerSnapshotBuffer;
4850

4951
private final int kafkaTxnProducerStateTopicSnapshotIntervalSeconds;
52+
private final int kafkaTxnPurgeAbortedTxnIntervalSeconds;
5053

5154
private volatile long mapEndOffset = -1;
5255

5356
private long lastSnapshotTime;
57+
private long lastPurgeAbortedTxnTime;
5458

59+
private volatile long abortedTxnsPurgeOffset = -1;
5560

5661
public ProducerStateManager(String topicPartition,
5762
String kafkaTopicUUID,
5863
ProducerStateManagerSnapshotBuffer producerStateManagerSnapshotBuffer,
59-
int kafkaTxnProducerStateTopicSnapshotIntervalSeconds) {
64+
int kafkaTxnProducerStateTopicSnapshotIntervalSeconds,
65+
int kafkaTxnPurgeAbortedTxnIntervalSeconds) {
6066
this.topicPartition = topicPartition;
6167
this.kafkaTopicUUID = kafkaTopicUUID;
6268
this.producerStateManagerSnapshotBuffer = producerStateManagerSnapshotBuffer;
6369
this.kafkaTxnProducerStateTopicSnapshotIntervalSeconds = kafkaTxnProducerStateTopicSnapshotIntervalSeconds;
70+
this.kafkaTxnPurgeAbortedTxnIntervalSeconds = kafkaTxnPurgeAbortedTxnIntervalSeconds;
6471
this.lastSnapshotTime = System.currentTimeMillis();
72+
this.lastPurgeAbortedTxnTime = System.currentTimeMillis();
6573
}
6674

6775
public CompletableFuture<Void> recover(PartitionLog partitionLog, Executor executor) {
@@ -159,6 +167,34 @@ void maybeTakeSnapshot(Executor executor) {
159167
takeSnapshot(executor);
160168
}
161169

170+
void updateAbortedTxnsPurgeOffset(long abortedTxnsPurgeOffset) {
171+
if (log.isDebugEnabled()) {
172+
log.debug("{} updateAbortedTxnsPurgeOffset {}", topicPartition, abortedTxnsPurgeOffset);
173+
}
174+
if (abortedTxnsPurgeOffset < 0) {
175+
return;
176+
}
177+
this.abortedTxnsPurgeOffset = abortedTxnsPurgeOffset;
178+
}
179+
180+
long maybePurgeAbortedTx() {
181+
if (mapEndOffset == -1) {
182+
return 0;
183+
}
184+
long now = System.currentTimeMillis();
185+
long deltaFromLast = (now - lastPurgeAbortedTxnTime) / 1000;
186+
if (deltaFromLast / 1000 <= kafkaTxnPurgeAbortedTxnIntervalSeconds) {
187+
return 0;
188+
}
189+
lastPurgeAbortedTxnTime = now;
190+
return executePurgeAbortedTx();
191+
}
192+
193+
@VisibleForTesting
194+
long executePurgeAbortedTx() {
195+
return purgeAbortedTxns(abortedTxnsPurgeOffset);
196+
}
197+
162198
private ProducerStateManagerSnapshot getProducerStateManagerSnapshot() {
163199
ProducerStateManagerSnapshot snapshot;
164200
synchronized (abortedIndexList) {
@@ -270,15 +306,39 @@ public void completeTxn(CompletedTxn completedTxn) {
270306
}
271307
}
272308

309+
public boolean hasSomeAbortedTransactions() {
310+
return !abortedIndexList.isEmpty();
311+
}
312+
313+
public long purgeAbortedTxns(long offset) {
314+
AtomicLong count = new AtomicLong();
315+
synchronized (abortedIndexList) {
316+
abortedIndexList.removeIf(tx -> {
317+
boolean toRemove = tx.lastOffset() < offset;
318+
if (toRemove) {
319+
log.info("Transaction {} can be removed (lastOffset {} < {})", tx, tx.lastOffset(), offset);
320+
count.incrementAndGet();
321+
}
322+
return toRemove;
323+
});
324+
if (!abortedIndexList.isEmpty()) {
325+
log.info("There are still {} aborted tx on {}", abortedIndexList.size(), topicPartition);
326+
}
327+
}
328+
return count.get();
329+
}
330+
273331
public List<FetchResponse.AbortedTransaction> getAbortedIndexList(long fetchOffset) {
274-
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
275-
for (AbortedTxn abortedTxn : abortedIndexList) {
276-
if (abortedTxn.lastOffset() >= fetchOffset) {
277-
abortedTransactions.add(
278-
new FetchResponse.AbortedTransaction(abortedTxn.producerId(), abortedTxn.firstOffset()));
332+
synchronized (abortedIndexList) {
333+
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
334+
for (AbortedTxn abortedTxn : abortedIndexList) {
335+
if (abortedTxn.lastOffset() >= fetchOffset) {
336+
abortedTransactions.add(
337+
new FetchResponse.AbortedTransaction(abortedTxn.producerId(), abortedTxn.firstOffset()));
338+
}
279339
}
340+
return abortedTransactions;
280341
}
281-
return abortedTransactions;
282342
}
283343

284344
public void handleMissingDataBeforeRecovery(long minOffset, long snapshotOffset) {

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/storage/ReplicaManager.java

+4
Original file line numberDiff line numberDiff line change
@@ -337,4 +337,8 @@ public void tryCompleteDelayedFetch(DelayedOperationKey key) {
337337
}
338338
}
339339

340+
public CompletableFuture<?> updatePurgeAbortedTxnsOffsets() {
341+
return logManager.updatePurgeAbortedTxnsOffsets();
342+
}
343+
340344
}

0 commit comments

Comments
 (0)