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

Commit

Permalink
[branch-2.11][transaction] Implement producer state manager recovery (#…
Browse files Browse the repository at this point in the history
…1923)

### Motivation

Cherry-pick s4k transaction producer state manager snapshot recovery
feature.

### Modifications

* Support producer state manager recovery

Co-authored-by: Enrico Olivelli <[email protected]>
Co-authored-by: Michael Marshall <[email protected]>
(cherry picked from commit fcf5234)

Fix conflicts caused by #1759
  • Loading branch information
Demogorgon314 authored and BewareMyPower committed Jul 17, 2023
1 parent 3ba7e21 commit 24d805d
Show file tree
Hide file tree
Showing 35 changed files with 1,905 additions and 420 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -107,10 +107,12 @@ public boolean tryComplete() {
return true;
}
for (Map.Entry<TopicPartition, PartitionLog.ReadRecordsResult> entry : readRecordsResult.entrySet()) {
TopicPartition tp = entry.getKey();
PartitionLog.ReadRecordsResult result = entry.getValue();
PartitionLog partitionLog = replicaManager.getPartitionLog(tp, context.getNamespacePrefix());
PositionImpl currLastPosition = (PositionImpl) partitionLog.getLastPosition(context.getTopicManager());
PartitionLog partitionLog = result.partitionLog();
if (partitionLog == null) {
return true;
}
PositionImpl currLastPosition = (PositionImpl) partitionLog.getLastPosition();
if (currLastPosition.compareTo(PositionImpl.EARLIEST) == 0) {
HAS_ERROR_UPDATER.set(this, true);
return forceComplete();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import io.streamnative.pulsar.handlers.kop.schemaregistry.SchemaRegistryChannelInitializer;
import io.streamnative.pulsar.handlers.kop.stats.PrometheusMetricsProvider;
import io.streamnative.pulsar.handlers.kop.stats.StatsLogger;
import io.streamnative.pulsar.handlers.kop.storage.MemoryProducerStateManagerSnapshotBuffer;
import io.streamnative.pulsar.handlers.kop.storage.ProducerStateManagerSnapshotBuffer;
import io.streamnative.pulsar.handlers.kop.storage.ReplicaManager;
import io.streamnative.pulsar.handlers.kop.utils.ConfigurationUtils;
import io.streamnative.pulsar.handlers.kop.utils.KopTopic;
Expand All @@ -47,8 +49,10 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.common.util.OrderedExecutor;
import org.apache.bookkeeper.common.util.OrderedScheduler;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.configuration.PropertiesConfiguration;
Expand Down Expand Up @@ -87,6 +91,8 @@ public class KafkaProtocolHandler implements ProtocolHandler, TenantContextManag
private DelayedOperationPurgatory<DelayedOperation> producePurgatory;
private DelayedOperationPurgatory<DelayedOperation> fetchPurgatory;
private LookupClient lookupClient;

private KafkaTopicLookupService kafkaTopicLookupService;
@VisibleForTesting
@Getter
private Map<InetSocketAddress, ChannelInitializer<SocketChannel>> channelInitializerMap;
Expand All @@ -113,6 +119,9 @@ public class KafkaProtocolHandler implements ProtocolHandler, TenantContextManag
private final Map<String, GroupCoordinator> groupCoordinatorsByTenant = new ConcurrentHashMap<>();
private final Map<String, TransactionCoordinator> transactionCoordinatorByTenant = new ConcurrentHashMap<>();


private OrderedExecutor recoveryExecutor;

@Override
public GroupCoordinator getGroupCoordinator(String tenant) {
return groupCoordinatorsByTenant.computeIfAbsent(tenant, this::createAndBootGroupCoordinator);
Expand Down Expand Up @@ -256,6 +265,12 @@ private void invalidatePartitionLog(TopicName topicName) {
});
namespaceService.addNamespaceBundleOwnershipListener(bundleListener);

recoveryExecutor = OrderedExecutor
.newBuilder()
.name("kafka-tx-recovery")
.numThreads(kafkaConfig.getKafkaTransactionRecoveryNumThreads())
.build();

if (kafkaConfig.isKafkaManageSystemNamespaces()) {
// initialize default Group Coordinator
getGroupCoordinator(kafkaConfig.getKafkaMetadataTenant());
Expand Down Expand Up @@ -411,6 +426,20 @@ private KafkaChannelInitializer newKafkaChannelInitializer(final EndPoint endPoi
lookupClient);
}

class ProducerStateManagerSnapshotProvider implements Function<String, ProducerStateManagerSnapshotBuffer> {
@Override
public ProducerStateManagerSnapshotBuffer apply(String tenant) {
if (!kafkaConfig.isKafkaTransactionCoordinatorEnabled()) {
return new MemoryProducerStateManagerSnapshotBuffer();
}
return getTransactionCoordinator(tenant)
.getProducerStateManagerSnapshotBuffer();
}
}

private Function<String, ProducerStateManagerSnapshotBuffer> getProducerStateManagerSnapshotBufferByTenant =
new ProducerStateManagerSnapshotProvider();

// this is called after initialize, and with kafkaConfig, brokerService all set.
@Override
public Map<InetSocketAddress, ChannelInitializer<SocketChannel>> newChannelInitializers() {
Expand All @@ -426,13 +455,19 @@ public Map<InetSocketAddress, ChannelInitializer<SocketChannel>> newChannelIniti
.timeoutTimer(SystemTimer.builder().executorName("fetch").build())
.build();

kafkaTopicLookupService = new KafkaTopicLookupService(brokerService);

replicaManager = new ReplicaManager(
kafkaConfig,
requestStats,
Time.SYSTEM,
brokerService.getEntryFilters(),
producePurgatory,
fetchPurgatory);
fetchPurgatory,
kafkaTopicLookupService,
getProducerStateManagerSnapshotBufferByTenant,
recoveryExecutor
);

try {
ImmutableMap.Builder<InetSocketAddress, ChannelInitializer<SocketChannel>> builder =
Expand Down Expand Up @@ -480,6 +515,17 @@ public void close() {
statsProvider.stop();
sendResponseScheduler.shutdown();

if (offsetTopicClient != null) {
offsetTopicClient.close();
}
if (txnTopicClient != null) {
txnTopicClient.close();
}
if (adminManager != null) {
adminManager.shutdown();
}
recoveryExecutor.shutdown();

List<CompletableFuture<?>> closeHandles = new ArrayList<>();
if (offsetTopicClient != null) {
closeHandles.add(offsetTopicClient.closeAsync());
Expand Down Expand Up @@ -571,6 +617,8 @@ public TransactionCoordinator initTransactionCoordinator(String tenant, PulsarAd
.transactionLogNumPartitions(kafkaConfig.getKafkaTxnLogTopicNumPartitions())
.transactionMetadataTopicName(MetadataUtils.constructTxnLogTopicBaseName(tenant, kafkaConfig))
.transactionProducerIdTopicName(MetadataUtils.constructTxnProducerIdTopicBaseName(tenant, kafkaConfig))
.transactionProducerStateSnapshotTopicName(MetadataUtils.constructTxProducerStateTopicBaseName(tenant,
kafkaConfig))
.abortTimedOutTransactionsIntervalMs(kafkaConfig.getKafkaTxnAbortTimedOutTransactionCleanupIntervalMs())
.transactionalIdExpirationMs(kafkaConfig.getKafkaTransactionalIdExpirationMs())
.removeExpiredTransactionalIdsIntervalMs(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1754,7 +1754,6 @@ protected void handleJoinGroupRequest(KafkaHeaderAndRequest joinGroup,
joinGroupResult.getLeaderId(),
members
);

if (log.isTraceEnabled()) {
log.trace("Sending join group response {} for correlation id {} to client {}.",
response, joinGroup.getHeader().correlationId(), joinGroup.getHeader().clientId());
Expand Down Expand Up @@ -2151,6 +2150,10 @@ protected void handleInitProducerId(KafkaHeaderAndRequest kafkaHeaderAndRequest,
.setErrorCode(resp.getError().code())
.setProducerId(resp.getProducerId())
.setProducerEpoch(resp.getProducerEpoch());
if (resp.getError() == Errors.COORDINATOR_LOAD_IN_PROGRESS
|| resp.getError() == Errors.CONCURRENT_TRANSACTIONS) {
responseData.setThrottleTimeMs(1000);
}
response.complete(new InitProducerIdResponse(responseData));
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public class KafkaServiceConfiguration extends ServiceConfiguration {
private static final int OffsetsMessageTTL = 3 * 24 * 3600;
// txn configuration
public static final int DefaultTxnLogTopicNumPartitions = 50;
public static final int DefaultTxnProducerStateLogTopicNumPartitions = 8;
public static final int DefaultTxnCoordinatorSchedulerNum = 1;
public static final int DefaultTxnStateManagerSchedulerNum = 1;
public static final long DefaultAbortTimedOutTransactionsIntervalMs = TimeUnit.SECONDS.toMillis(10);
Expand Down Expand Up @@ -425,6 +426,24 @@ public class KafkaServiceConfiguration extends ServiceConfiguration {
)
private int kafkaTxnLogTopicNumPartitions = DefaultTxnLogTopicNumPartitions;

@FieldContext(
category = CATEGORY_KOP_TRANSACTION,
doc = "Number of partitions for the transaction producer state topic"
)
private int kafkaTxnProducerStateTopicNumPartitions = DefaultTxnProducerStateLogTopicNumPartitions;

@FieldContext(
category = CATEGORY_KOP_TRANSACTION,
doc = "Interval for taking snapshots of the status of pending transactions"
)
private int kafkaTxnProducerStateTopicSnapshotIntervalSeconds = 300;

@FieldContext(
category = CATEGORY_KOP_TRANSACTION,
doc = "Number of threads dedicated to transaction recovery"
)
private int kafkaTransactionRecoveryNumThreads = 8;

@FieldContext(
category = CATEGORY_KOP_TRANSACTION,
doc = "The interval in milliseconds at which to rollback transactions that have timed out."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@
public class KafkaTopicConsumerManager implements Closeable {

private final PersistentTopic topic;
private final KafkaRequestHandler requestHandler;

private final AtomicBoolean closed = new AtomicBoolean(false);

Expand All @@ -67,13 +66,21 @@ public class KafkaTopicConsumerManager implements Closeable {

private final boolean skipMessagesWithoutIndex;

private final String description;

KafkaTopicConsumerManager(KafkaRequestHandler requestHandler, PersistentTopic topic) {
this(requestHandler.ctx.channel() + "",
requestHandler.isSkipMessagesWithoutIndex(),
topic);
}

public KafkaTopicConsumerManager(String description, boolean skipMessagesWithoutIndex, PersistentTopic topic) {
this.topic = topic;
this.cursors = new ConcurrentHashMap<>();
this.createdCursors = new ConcurrentHashMap<>();
this.lastAccessTimes = new ConcurrentHashMap<>();
this.requestHandler = requestHandler;
this.skipMessagesWithoutIndex = requestHandler.isSkipMessagesWithoutIndex();
this.description = description;
this.skipMessagesWithoutIndex = skipMessagesWithoutIndex;
}

// delete expired cursors, so backlog can be cleared.
Expand All @@ -96,7 +103,7 @@ void deleteOneExpiredCursor(long offset) {
if (cursorFuture != null) {
if (log.isDebugEnabled()) {
log.debug("[{}] Cursor timed out for offset: {}, cursors cache size: {}",
requestHandler.ctx.channel(), offset, cursors.size());
description, offset, cursors.size());
}

// TODO: Should we just cancel this future?
Expand All @@ -118,14 +125,19 @@ public void deleteOneCursorAsync(ManagedCursor cursor, String reason) {
public void deleteCursorComplete(Object ctx) {
if (log.isDebugEnabled()) {
log.debug("[{}] Cursor {} for topic {} deleted successfully for reason: {}.",
requestHandler.ctx.channel(), cursor.getName(), topic.getName(), reason);
description, cursor.getName(), topic.getName(), reason);
}
}

@Override
public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) {
log.warn("[{}] Error deleting cursor {} for topic {} for reason: {}.",
requestHandler.ctx.channel(), cursor.getName(), topic.getName(), reason, exception);
if (exception instanceof ManagedLedgerException.CursorNotFoundException) {
log.debug("[{}] Cursor already deleted {} for topic {} for reason: {} - {}.",
description, cursor.getName(), topic.getName(), reason, exception.toString());
} else {
log.warn("[{}] Error deleting cursor {} for topic {} for reason: {}.",
description, cursor.getName(), topic.getName(), reason, exception);
}
}
}, null);
createdCursors.remove(cursor.getName());
Expand All @@ -148,7 +160,7 @@ public CompletableFuture<Pair<ManagedCursor, Long>> removeCursorFuture(long offs

if (log.isDebugEnabled()) {
log.debug("[{}] Get cursor for offset: {} in cache. cache size: {}",
requestHandler.ctx.channel(), offset, cursors.size());
description, offset, cursors.size());
}
return cursorFuture;
}
Expand Down Expand Up @@ -182,7 +194,7 @@ public void add(long offset, Pair<ManagedCursor, Long> pair) {

if (log.isDebugEnabled()) {
log.debug("[{}] Add cursor back {} for offset: {}",
requestHandler.ctx.channel(), pair.getLeft().getName(), offset);
description, pair.getLeft().getName(), offset);
}
}

Expand All @@ -194,7 +206,7 @@ public void close() {
}
if (log.isDebugEnabled()) {
log.debug("[{}] Close TCM for topic {}.",
requestHandler.ctx.channel(), topic.getName());
description, topic.getName());
}
final List<CompletableFuture<Pair<ManagedCursor, Long>>> cursorFuturesToClose = new ArrayList<>();
cursors.forEach((ignored, cursorFuture) -> cursorFuturesToClose.add(cursorFuture));
Expand Down Expand Up @@ -224,7 +236,7 @@ private CompletableFuture<Pair<ManagedCursor, Long>> asyncGetCursorByOffset(long
if (((ManagedLedgerImpl) ledger).getState() == ManagedLedgerImpl.State.Closed) {
log.error("[{}] Async get cursor for offset {} for topic {} failed, "
+ "because current managedLedger has been closed",
requestHandler.ctx.channel(), offset, topic.getName());
description, offset, topic.getName());
CompletableFuture<Pair<ManagedCursor, Long>> future = new CompletableFuture<>();
future.completeExceptionally(new Exception("Current managedLedger for "
+ topic.getName() + " has been closed."));
Expand All @@ -243,7 +255,7 @@ private CompletableFuture<Pair<ManagedCursor, Long>> asyncGetCursorByOffset(long
final PositionImpl previous = ((ManagedLedgerImpl) ledger).getPreviousPosition((PositionImpl) position);
if (log.isDebugEnabled()) {
log.debug("[{}] Create cursor {} for offset: {}. position: {}, previousPosition: {}",
requestHandler.ctx.channel(), cursorName, offset, position, previous);
description, cursorName, offset, position, previous);
}
try {
final ManagedCursor newCursor = ledger.newNonDurableCursor(previous, cursorName);
Expand All @@ -252,7 +264,7 @@ private CompletableFuture<Pair<ManagedCursor, Long>> asyncGetCursorByOffset(long
return Pair.of(newCursor, offset);
} catch (ManagedLedgerException e) {
log.error("[{}] Error new cursor for topic {} at offset {} - {}. will cause fetch data error.",
requestHandler.ctx.channel(), topic.getName(), offset, previous, e);
description, topic.getName(), offset, previous, e);
return null;
}
});
Expand Down
Loading

0 comments on commit 24d805d

Please sign in to comment.