Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix no snapshot for normal message #61

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1583,67 +1583,69 @@ protected void handleProducer(final CommandProducer cmdProducer) {
});

schemaVersionFuture.thenAccept(schemaVersion -> {
topic.checkIfTransactionBufferRecoverCompletely(isTxnEnabled).thenAccept(future -> {
CompletionStage<Subscription> createInitSubFuture;
if (!Strings.isNullOrEmpty(initialSubscriptionName)
&& topic.isPersistent()
&& !topic.getSubscriptions().containsKey(initialSubscriptionName)) {
createInitSubFuture = service.isAllowAutoSubscriptionCreationAsync(topicName)
.thenCompose(isAllowAutoSubscriptionCreation -> {
if (!isAllowAutoSubscriptionCreation) {
return CompletableFuture.failedFuture(
new BrokerServiceException.NotAllowedException(
"Could not create the initial subscription due to"
+ " the auto subscription creation is not allowed."));
}
return topic.createSubscription(initialSubscriptionName,
InitialPosition.Earliest, false, null);
});
} else {
createInitSubFuture = CompletableFuture.completedFuture(null);
}
topic.checkIfTransactionBufferRecoverCompletely(isTxnEnabled)
.thenCompose(future -> topic.takeFirstSnapshotIfNeed(isTxnEnabled))
.thenAccept(future -> {
CompletionStage<Subscription> createInitSubFuture;
if (!Strings.isNullOrEmpty(initialSubscriptionName)
&& topic.isPersistent()
&& !topic.getSubscriptions().containsKey(initialSubscriptionName)) {
createInitSubFuture = service.isAllowAutoSubscriptionCreationAsync(topicName)
.thenCompose(isAllowAutoSubscriptionCreation -> {
if (!isAllowAutoSubscriptionCreation) {
return CompletableFuture.failedFuture(
new BrokerServiceException.NotAllowedException(
"Could not create the initial subscription due to"
+ " the auto subscription creation is not allowed."));
}
return topic.createSubscription(initialSubscriptionName,
InitialPosition.Earliest, false, null);
});
} else {
createInitSubFuture = CompletableFuture.completedFuture(null);
}

createInitSubFuture.whenComplete((sub, ex) -> {
if (ex != null) {
final Throwable rc = FutureUtil.unwrapCompletionException(ex);
if (rc instanceof BrokerServiceException.NotAllowedException) {
log.warn("[{}] {} initialSubscriptionName: {}, topic: {}",
remoteAddress, rc.getMessage(), initialSubscriptionName, topicName);
if (producerFuture.completeExceptionally(rc)) {
commandSender.sendErrorResponse(requestId,
ServerError.NotAllowedError, rc.getMessage());
createInitSubFuture.whenComplete((sub, ex) -> {
if (ex != null) {
final Throwable rc = FutureUtil.unwrapCompletionException(ex);
if (rc instanceof BrokerServiceException.NotAllowedException) {
log.warn("[{}] {} initialSubscriptionName: {}, topic: {}",
remoteAddress, rc.getMessage(), initialSubscriptionName, topicName);
if (producerFuture.completeExceptionally(rc)) {
commandSender.sendErrorResponse(requestId,
ServerError.NotAllowedError, rc.getMessage());
}
producers.remove(producerId, producerFuture);
return;
}
String msg =
"Failed to create the initial subscription: " + ex.getCause().getMessage();
log.warn("[{}] {} initialSubscriptionName: {}, topic: {}",
remoteAddress, msg, initialSubscriptionName, topicName);
if (producerFuture.completeExceptionally(ex)) {
commandSender.sendErrorResponse(requestId,
BrokerServiceException.getClientErrorCode(ex), msg);
}
producers.remove(producerId, producerFuture);
return;
}
producers.remove(producerId, producerFuture);
return;
}
String msg =
"Failed to create the initial subscription: " + ex.getCause().getMessage();
log.warn("[{}] {} initialSubscriptionName: {}, topic: {}",
remoteAddress, msg, initialSubscriptionName, topicName);
if (producerFuture.completeExceptionally(ex)) {

buildProducerAndAddTopic(topic, producerId, producerName, requestId, isEncrypted,
metadata, schemaVersion, epoch, userProvidedProducerName, topicName,
producerAccessMode, topicEpoch, supportsPartialProducer, producerFuture);
});
}).exceptionally(exception -> {
Throwable cause = exception.getCause();
log.error("producerId {}, requestId {} : TransactionBuffer recover failed",
producerId, requestId, exception);
if (producerFuture.completeExceptionally(exception)) {
commandSender.sendErrorResponse(requestId,
BrokerServiceException.getClientErrorCode(ex), msg);
ServiceUnitNotReadyException.getClientErrorCode(cause),
cause.getMessage());
}
producers.remove(producerId, producerFuture);
return;
}

buildProducerAndAddTopic(topic, producerId, producerName, requestId, isEncrypted,
metadata, schemaVersion, epoch, userProvidedProducerName, topicName,
producerAccessMode, topicEpoch, supportsPartialProducer, producerFuture);
});
}).exceptionally(exception -> {
Throwable cause = exception.getCause();
log.error("producerId {}, requestId {} : TransactionBuffer recover failed",
producerId, requestId, exception);
if (producerFuture.completeExceptionally(exception)) {
commandSender.sendErrorResponse(requestId,
ServiceUnitNotReadyException.getClientErrorCode(cause),
cause.getMessage());
}
producers.remove(producerId, producerFuture);
return null;
});
return null;
});
});
});
return backlogQuotaCheckFuture;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,13 @@ default void setEntryTimestamp(long entryTimestamp) {
*/
CompletableFuture<Void> checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled);

/**
* Take snapshot if needed.
* @param isTxnEnabled isTxnEnabled
* @return a future represents the result of take snapshot operation.
*/
CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean isTxnEnabled);

/**
* record add-latency.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,6 +261,11 @@ public CompletableFuture<Void> checkIfTransactionBufferRecoverCompletely(boolean
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean enableTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<Consumer> subscribe(SubscriptionOption option) {
return internalSubscribe(option.getCnx(), option.getSubscriptionName(), option.getConsumerId(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -845,6 +845,11 @@ public CompletableFuture<Void> checkIfTransactionBufferRecoverCompletely(boolean
return getTransactionBuffer().checkIfTBRecoverCompletely(isTxnEnabled);
}

@Override
public CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean isTxnEnabled) {
return getTransactionBuffer().takeFirstSnapshotIfNeed(isTxnEnabled);
}

@Override
protected CompletableFuture<Long> incrementTopicEpoch(Optional<Long> currentEpoch) {
long newEpoch = currentEpoch.orElse(-1L) + 1;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,12 @@ public interface TransactionBuffer {
*/
CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxn);


/**
* Take snapshot if needed.
* @param enableTxn
* @return a future represents the result of take snapshot operation.
*/
CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean enableTxn);

long getOngoingTxnCount();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,11 @@ public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean enableTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public long getOngoingTxnCount() {
return this.buffers.values().stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -219,20 +219,11 @@ public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxnEnabled)
} else {
CompletableFuture<Void> completableFuture = new CompletableFuture<>();
transactionBufferFuture.thenRun(() -> {
if (checkIfNoSnapshot()) {
snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(maxReadPosition).thenRun(() -> {
if (changeToReadyStateFromNoSnapshot()) {
timer.newTimeout(TopicTransactionBuffer.this,
takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
}
completableFuture.complete(null);
}).exceptionally(exception -> {
log.error("Topic {} failed to take snapshot", this.topic.getName());
completableFuture.completeExceptionally(exception);
return null;
});
} else {
if (checkIfNoSnapshot() || checkIfReady()) {
thetumbled marked this conversation as resolved.
Show resolved Hide resolved
completableFuture.complete(null);
} else {
completableFuture.completeExceptionally(new BrokerServiceException
.ServiceUnitNotReadyException("TransactionBuffer recover failed"));
}
}).exceptionally(exception -> {
log.error("Topic {}: TransactionBuffer recover failed", this.topic.getName(), exception.getCause());
Expand All @@ -243,6 +234,19 @@ public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxnEnabled)
}
}

@Override
public CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean enableTxn) {
CompletableFuture<Void> completableFuture = new CompletableFuture<>();
if (enableTxn && checkIfNoSnapshot()) {
this.snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(maxReadPosition)
.thenRun(() -> changeToReadyStateFromNoSnapshot())
.thenAccept(__ -> completableFuture.complete(null));
} else {
completableFuture.complete(null);
}
return completableFuture;
}

@Override
public long getOngoingTxnCount() {
return this.ongoingTxns.size();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,11 @@ public CompletableFuture<Void> checkIfTBRecoverCompletely(boolean isTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public CompletableFuture<Void> takeFirstSnapshotIfNeed(boolean enableTxn) {
return CompletableFuture.completedFuture(null);
}

@Override
public long getOngoingTxnCount() {
return 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -343,6 +343,48 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception {
assertGetLastMessageId(consumer, expectedLastMessageID1);
}

@Test
public void testNormalProductionNoSnapshot() throws Exception {
String topic = "persistent://" + NAMESPACE1 + "/testNormalProductionNoSnapshot";
@Cleanup
Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topic)
.create();

PersistentTopic topicRef = null;
for (int i = 0; i < pulsarServiceList.size(); i++) {
try {
topicRef = (PersistentTopic) pulsarServiceList.get(i)
.getBrokerService().getTopic(topic, true).get().get();
break;
} catch (Exception e) {
}
}
TopicTransactionBuffer transactionBuffer = (TopicTransactionBuffer) topicRef.getTransactionBuffer();

// check the transaction buffer state is NoSnapshot
Assert.assertEquals(transactionBuffer.getState(), TopicTransactionBufferState.State.NoSnapshot);

// send 5 original messages.
for (int i = 0; i < 5; i++) {
producer.newMessage().send();
}

// check the transaction buffer state is NoSnapshot
Assert.assertEquals(transactionBuffer.getState(), TopicTransactionBufferState.State.NoSnapshot);

// create consumer to get last message id, trigger the snapshot
@Cleanup
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.subscriptionName("sub")
.subscribe();
consumer.getLastMessageIds();

// check the transaction buffer state is Ready
Assert.assertEquals(transactionBuffer.getState(), TopicTransactionBufferState.State.NoSnapshot);
}

/**
* produce 3 messages and then trigger a ledger switch,
* then create a transaction and send a transactional message.
Expand Down
Loading