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

[pulsar-client] fix deadlock on send failure #6488

Merged
merged 1 commit into from
Mar 6, 2020
Merged
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 @@ -77,6 +77,7 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.impl.ClientCnx;
import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.MessageCrypto;
import org.apache.pulsar.client.impl.MessageIdImpl;
Expand Down Expand Up @@ -3283,4 +3284,36 @@ public void testConsumerStartMessageIdAtExpectedPos(boolean batching, boolean st
consumer.close();
producer.close();
}

/**
* It verifies that message failure successfully releases semaphore and client successfully receives
* InvalidMessageException.
*
* @throws Exception
*/
@Test
public void testReleaseSemaphoreOnFailMessages() throws Exception {
log.info("-- Starting {} test --", methodName);

int maxPendingMessages = 10;
ProducerBuilder<byte[]> producerBuilder = pulsarClient.newProducer().enableBatching(false)
.blockIfQueueFull(true).maxPendingMessages(maxPendingMessages)
.topic("persistent://my-property/my-ns/my-topic2");

Producer<byte[]> producer = producerBuilder.create();
List<Future<MessageId>> futures = Lists.newArrayList();

// Asynchronously produce messages
byte[] message = new byte[ClientCnx.getMaxMessageSize() + 1];
for (int i = 0; i < maxPendingMessages + 10; i++) {
Future<MessageId> future = producer.sendAsync(message);
try {
future.get();
fail("should fail with InvalidMessageException");
} catch (Exception e) {
assertTrue(e.getCause() instanceof PulsarClientException.InvalidMessageException);
}
}
log.info("-- Exiting {} test --", methodName);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -351,7 +351,7 @@ public void sendAsync(Message<?> message, SendCallback callback) {
PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds %d bytes",
producerName, topic, compressedStr, compressedSize, ClientCnx.getMaxMessageSize()));
callback.sendComplete(invalidMessageException);
completeCallbackAndReleaseSemaphore(callback, invalidMessageException);
return;
}
}
Expand All @@ -360,7 +360,7 @@ public void sendAsync(Message<?> message, SendCallback callback) {
PulsarClientException.InvalidMessageException invalidMessageException =
new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s can not reuse the same message", producerName, topic));
callback.sendComplete(invalidMessageException);
completeCallbackAndReleaseSemaphore(callback, invalidMessageException);
compressedPayload.release();
return;
}
Expand Down Expand Up @@ -455,11 +455,9 @@ public void sendAsync(Message<?> message, SendCallback callback) {
}
}
} catch (PulsarClientException e) {
semaphore.release();
callback.sendComplete(e);
completeCallbackAndReleaseSemaphore(callback, e);
} catch (Throwable t) {
semaphore.release();
callback.sendComplete(new PulsarClientException(t));
completeCallbackAndReleaseSemaphore(callback, new PulsarClientException(t));
}
}

Expand All @@ -471,8 +469,9 @@ private boolean populateMessageSchema(MessageImpl msg, SendCallback callback) {
return true;
}
if (!isMultiSchemaEnabled(true)) {
callback.sendComplete(new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s is disabled the `MultiSchema`", producerName, topic)));
PulsarClientException.InvalidMessageException e = new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s is disabled the `MultiSchema`", producerName, topic));
completeCallbackAndReleaseSemaphore(callback, e);
return false;
}
SchemaHash schemaHash = SchemaHash.of(msg.getSchema());
Expand Down Expand Up @@ -872,6 +871,11 @@ private void releaseSemaphoreForSendOp(OpSendMsg op) {
semaphore.release(isBatchMessagingEnabled() ? op.numMessagesInBatch : 1);
}

private void completeCallbackAndReleaseSemaphore(SendCallback callback, Exception exception) {
semaphore.release();
callback.sendComplete(exception);
}

/**
* Checks message checksum to retry if message was corrupted while sending to broker. Recomputes checksum of the
* message header-payload again.
Expand Down