-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
PIP-2: Introduce non-persistent topics #538
Conversation
f258517
to
db8e757
Compare
This commit is trying to do more than just making broker configurable to own non persistent topics. Or is it due to PR #452 is not merged yet? |
fail("topic loading should have failed"); | ||
} catch (Exception e) { | ||
// Ok | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't it attempt to create persistent topic and ensure it is ok?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
actually in this test we start broker in non-persistent
mode and we try to load persistent
topic which will fail because load-manager doesn't find any broker which support persistent-topic.
However, producer retries to do lookup and recreate but here, we have put the timeout on producer-creation which will timeout and completes the test.
pulsarClient.createProducerAsync(topicName, producerConf).get(1, TimeUnit.SECONDS);
} | ||
|
||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we have a case where both type is enabled and topic creation succeeds?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we have two testcases: one with non-persistent enabled broker other one without it. In other cases broker has default configuration to load both types of topics.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Change looks good to me. I'd like to see a bit more of code sharing with the persistent
version of the dispatcher/subscription/replicator.
@@ -42,6 +42,9 @@ | |||
private final String webServiceUrlTls; | |||
private final String pulsarServiceUrl; | |||
private final String pulsarServiceUrlTls; | |||
private boolean enablePersistentTopics=true; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Minor and probably a bit of nit-picking. Here it should be referring on wether the feature is "enabled" or "disabled", rather than be imperative ("enable").
Should we rename to persistentTopicsEnabled
, nonPersistentTopicsEnabled
?
@@ -35,6 +35,9 @@ | |||
private final NamespaceName nsname; | |||
private final Range<Long> keyRange; | |||
private final NamespaceBundleFactory factory; | |||
// TODO: remove this once we remove broker persistent/non-persistent mode configuration |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you create a github issue for the TODO and mention it here?
@@ -236,5 +238,9 @@ public int compareTo(EntryCache other) { | |||
|
|||
} | |||
|
|||
public static Entry create(long ledgerId, long entryId, ByteBuf data) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this change still needed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, for non-persistent topics we want to create Entry
with specific ledgerId-entryId and payload so, it can be pass to other entities (eg: dispatcher, consumer) as those entities APIs requires a Entry
wrapper to pass payload/msgId information.
@@ -103,6 +109,8 @@ protected String domain() { | |||
return "topic"; | |||
} else if (uri.getPath().startsWith("persistent/")) { | |||
return "persistent"; | |||
} else if (uri.getPath().startsWith("non-persistent/")) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Side note: should we remove queue
and topic
here?
return TOTAL_AVAILABLE_PERMITS_UPDATER.get(this) > 0; | ||
} | ||
|
||
private Consumer getNextConsumer() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can this logic be shared with the persistent version of the dispatcher?
(Maybe using an abstract class?)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, I will again visit it to share things across both the persistent/non-persistent entitites.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Great, you can introduce some base class like AbstractDispatcherMultipleConsumers
with the common logic.
ACTIVE_CONSUMER_UPDATER.set(this, null); | ||
} | ||
|
||
private void pickAndScheduleActiveConsumer() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same here, most of these methods looks very similar to the "persistent" version. Share the same code if possible.
|
||
} | ||
|
||
public void sendMessage(Entry entry) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it possible to share any logic with the other replicator?
@rdhabalia One small issue I'm seeing on the CLI tool: $ bin/pulsar-admin non-persistent stats non-persistent://prop/cluster/ns/my-topic
Need to provide a persistent topic name
Usage: pulsar-admin non-persistent [options] [command] [command options]
Commands:
.... |
One more exception in broker :
|
yes, thanks for catching it. I will fix it.
Yes, yesterday, I have added commit to get |
175db00
to
f8a1192
Compare
@rdhabalia One test failure seems genuine :
|
Actually I had fixed it as part of last commit, it was failing becauseit was trying to get field-variable using reflection which was moved to I have addressed all your comments. However, I still have to add |
e35dba9
to
238f37c
Compare
238f37c
to
669e7a9
Compare
@rdhabalia I'm running a 1 prod - 1 cons test with non-persistent. Everything local. At the rate of 10K write/s, with no batching, I'm seeing the consumer is getting 1msg/s dropped. Ideally that should not happen since the system is not really overloaded. |
$ bin/pulsar-admin non-persistent stats non-persistent://prop/cluster/ns/my-topic
{
"msgRateIn" : 10000.319421984088,
"msgThroughputIn" : 1.0550336990193212E7,
"msgRateOut" : 9998.314769761299,
"msgThroughputOut" : 1.054822208209817E7,
"averageMsgSize" : 1055.0,
"storageSize" : 0,
"publishers" : [ {
"msgRateIn" : 10000.319421984088,
"msgThroughputIn" : 1.0550336990193212E7,
"averageMsgSize" : 1055.0,
"producerId" : 0,
"producerName" : "standalone-1-3",
"address" : "/127.0.0.1:51246",
"connectedSince" : "2017-07-27 11:10:49.446-0700",
"clientVersion" : "1.19-incubating-SNAPSHOT",
"msgDropRate" : 0.0
} ],
"subscriptions" : {
"sub" : {
"msgRateOut" : 9998.314769761299,
"msgThroughputOut" : 1.054822208209817E7,
"msgRateRedeliver" : 0.0,
"msgBacklog" : 0,
"blockedSubscriptionOnUnackedMsgs" : false,
"unackedMessages" : 0,
"type" : "Exclusive",
"msgRateExpired" : 0.0,
"consumers" : [ {
"msgRateOut" : 9998.314769761299,
"msgThroughputOut" : 1.054822208209817E7,
"msgRateRedeliver" : 0.0,
"consumerName" : "0ce8d",
"availablePermits" : 791,
"unackedMessages" : 0,
"blockedConsumerOnUnackedMsgs" : false,
"address" : "/127.0.0.1:51080",
"connectedSince" : "2017-07-27 11:07:36.213-0700",
"clientVersion" : "1.19-incubating-SNAPSHOT"
} ],
"msgDropRate" : 1.9501263804427469
}
},
"replication" : { },
"msgDropRate" : 0.0
} |
I feel it's happening due to not available permits? We drop the message for subscription in two cases:
In this case, it seems like consumer is not having enough permits? Let me run perf test again with debug log to confirm the behavior. |
Not sure, I increased to 50K receiver queue size and it still happens |
"consumerName" : "28513",
"availablePermits" : 417242,
"unackedMessages" : 0,
"blockedConsumerOnUnackedMsgs" : false,
"address" : "/127.0.0.1:52409",
"connectedSince" : "2017-07-27 11:33:51.270-0700",
"clientVersion" : "1.19-incubating-SNAPSHOT"
} ],
"msgDropRate" : 3.1666701082037405 |
This was even with 500K permits |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Change looks overall good to me. Just the concern about dropped messages in seemingly normal conditions.
Also, I would say to mark it as "Experimental" in the documentation, to set the right expectation, and notice that implementation details might still be changed in future releases.
@@ -775,15 +775,20 @@ public void skipAllMessages(@PathParam("property") String property, @PathParam(" | |||
} | |||
} else { | |||
validateAdminOperationOnDestination(dn, authoritative); | |||
PersistentTopic topic = getTopicReference(dn); | |||
if (!(getTopicReference(dn) instanceof PersistentTopic)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need to check if it's persistent topic? Shouldn't this already be guaranteed by having this handler on /admin/persistent/...
?
@@ -56,6 +58,8 @@ | |||
private final long producerId; | |||
private final String appId; | |||
private Rate msgIn; | |||
// it records msg-drop rate only for non-persistent topic | |||
private Rate msgDrop; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can this be final
? And maybe we can set to null
if it's persistent topic.
@@ -100,6 +102,8 @@ public ServerCnx(BrokerService service) { | |||
this.producers = new ConcurrentLongHashMap<>(8, 1); | |||
this.consumers = new ConcurrentLongHashMap<>(8, 1); | |||
this.replicatorPrefix = service.pulsar().getConfiguration().getReplicatorPrefix(); | |||
this.nonPersistentMessageSemaphore = new Semaphore( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All the events from the same connections are coming from same thread. Even when we write, we typically switch to the event loop thread for that connection.
Given that, the semaphore could just be a regular (non even volatile
variable` that we check and increment. No need for thread safety.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
so, you mean instead semaphore we can have just a counter like pendingSendRequest
, but then we need to decrement this counter on completedSendOperation()
and that called from a different thread.
so, it might not be a thread-safe and it might not throttle with accurate configured number? right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But the completedSendOperation()
is already called within the io thread, so an addition unlocked variable should be fine.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, for non-persistent topic it is been called from ordered-executor thread. So, counter will be incremented by io-thread and decrement by ordered-thread. Not, sure if it would be thread-safe in terms throttling with correct count.
so, do you still think it would be fine to keep one unlock counter variable would be fine here or it should be atomic?
if (producer.isNonPersistentTopic() && !nonPersistentMessageSemaphore.tryAcquire()) { | ||
final long producerId = send.getProducerId(); | ||
final long sequenceId = send.getSequenceId(); | ||
service.getTopicOrderedExecutor().submitOrdered(producer.getTopic(), SafeRun.safeRun(() -> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why schedule it in ordered thread? We are in the io thread, we should write it already from this thread.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
actually, we have to write it in ordered thread only else send-ack goes in random order and because of that client will not receive sequenceId in order and it will fail the pending msgs and close connection.
- if msg doesn't throttle here then broker processes it by delivering it to subscribers in ordered thread and after completion that ordered thread sends ack back to client.
- so, all the sending-ack operation has to be in ordered-thread only.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Uhm, but why wait for dispatching message to consumers before sending ack to producer?
I think the best option here is to send ack immediately to producer. We got the message in broker memory, that's the level of guarantee.
Later if the consumers are not ready, the message will be dropped.
if (--pendingSendRequest == ResumeReadsThreshold) { | ||
// Resume reading from socket | ||
ctx.channel().config().setAutoRead(true); | ||
} | ||
if (isNonPersistentTopic) { | ||
nonPersistentMessageSemaphore.release(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As I was mentioning above, the pendingSendRequest
is already a counter of pending requests from the producer and it's not synchronized or a semaphore.
@@ -46,7 +46,7 @@ | |||
|
|||
/** Client library version */ | |||
public String clientVersion; | |||
|
|||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
White spaces addition
@@ -57,7 +57,7 @@ | |||
|
|||
/** Timestamp of outbound connection establishment time */ | |||
public String outboundConnectedSince; | |||
|
|||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
White spaces addition
@@ -55,8 +55,8 @@ | |||
|
|||
/** List of connected consumers on this subscription w/ their stats */ | |||
public List<ConsumerStats> consumers; | |||
|
|||
public PersistentSubscriptionStats() { | |||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
White spaces addition
@rdhabalia One other note: If I enable batching the drop-rate falls to 0.016 /s though I suspect it's only counting the number of dropped batches |
after adding more debug logs where we record-msg drop: found out that all the message-drop happened due to connection is not writable. |
yes, we record msg-drop at dispatcher when consumer is not available/cnx-writable so, we don't deserialize message-metadata to get actual count. |
@rdhabalia One other thing is what happens when a subscriptions has no more consumers available. Right now the subscription stays around. Is that the intended behavior? I feel it's not really needed to be kept, since messages will be dropped anyway. |
60d3dd7
to
50fdd31
Compare
fa7fa9a
to
883af92
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 Nice work!
Motivation
Replacing #476 (due to rebase conflicts and namespace-configuration implementation) , and creating this PR on top of #452.
Modifications
Broker can be configured to start in a mode:
Load-manager gets topic-lookup request and assigns it to broker which supports that topic. Broker fails to load topics if it doesn't support that type of topic.
Result
Broker can start in a mode where it can serve only non-persistent topic.