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

Catastrophic frequent random subscription freezes, especially on high-traffic topics. #6054

Closed
devinbost opened this issue Jan 14, 2020 · 94 comments · Fixed by #10417
Closed
Labels
type/bug The PR fixed a bug or issue reported a bug

Comments

@devinbost
Copy link
Contributor

Describe the bug
Topics randomly freeze, causing catastrophic topic outages on a weekly (or more frequent) basis. This has been an issue as long as my team has used Pulsar, and it's been communicated to a number of folks on the Pulsar PMC committee.

(I thought an issue was already created for this bug, but I couldn't find it anywhere.)

To Reproduce
We have not figured out how to reproduce the issue. It's random (seems to be non-deterministic) and doesn't seem to have any clues in the broker logs.

Expected behavior
Topics should never just randomly stop working to where the only resolution is restarting the problem broker.

Steps to Diagnose and Temporarily Resolve
image
Step 2: Check the rate out on the topic. (click on the topic in the dashboard, or do a stats on the topic and look at the "msgRateOut")

If the rate out is 0 this is likely a frozen topic, but to verify do the following:

In the pulsar dashboard, click on the broker that topic is living on. If you see that there are multiple topic that have a rate out of 0, then proceed to the next step, if not it could potentially be another issue. Investigate further.
image

image

Step 3: Stop the broker on the server that the topic is living on. pulsar-broker stop .

Step 4: Wait for the backlog to be consumed and all the functions to be rescheduled. (typically wait for about 5-10 mins)

Environment:

Docker on bare metal running: `apachepulsar/pulsar-all:2.4.0`
on CentOS.
Brokers are the function workers. 

This has been an issue with previous versions of Pulsar as well.

Additional context

Problem was MUCH worse with Pulsar 2.4.2, so our team needed to roll back to 2.4.0 (which has the problem, but it's less frequent).
This is preventing the team from progressing in the use of Pulsar, and it's causing SLA problems with those who use our service.

@devinbost devinbost added the type/bug The PR fixed a bug or issue reported a bug label Jan 14, 2020
@cdbartholomew
Copy link
Contributor

FWIW, @devinbost I am suspicious of the netty version in the 2.4.X stream. There is a know memory usage issue (netty/netty#8814) in the 4.1.32 version of netty that is used in the stream. When I was testing, I would see issues like you are describing. Then I patched in the latest version of netty, and things seemed better. I have been running with a patched 2.4.2 for a while and have not seen any issues. You may be experiencing something completely different, but it might be worth trying 2.4.2 + latest netty.

@sijie
Copy link
Member

sijie commented Jan 15, 2020

@devinbost have you looked into permits when this issue happens?

@devinbost
Copy link
Contributor Author

@sijie We have, but I can't remember exactly what our findings were.
Is there anything in particular that we should look for regarding permits?

@devinbost
Copy link
Contributor Author

@cdbartholomew We actually started experiencing this issue before 2.4.0.

@devinbost
Copy link
Contributor Author

I noticed that each topic lives on a single broker, which creates a single point of failure.
Is there any interest in making topics higher availability? (I suppose it would be a workaround, but it might help prevent other issues.)

@sijie
Copy link
Member

sijie commented Jan 21, 2020

I noticed that each topic lives on a single broker, which creates a single point of failure.
Is there any interest in making topics higher availability?

We (StreamNative) have been helping folks from Tencent at developing a feature called ReadOnly brokers. It allows a topic can have multiple owners (one writeable owner and multiple readonly owners). It has been running on production for a while. They will contribute it back soon.

@sijie
Copy link
Member

sijie commented Jan 21, 2020

Is there anything in particular that we should look for regarding permits?

Incorrect permits has been one of the main reasons causing the consumer to be stalled. You can use "unload" command to unload a topic or a namespace bundle to trigger a consumer reconnect. It resets the consumer state to mitigate the problem.

@codelipenghui and @jiazhai are working a proposal on improving the permits protocol.

I am not sure if your problem is related to permits. but if the same problem occurs, the first thing you should do is to use topic stats or topic stats-internal to get the stats or internal stats of a topic.

@devinbost
Copy link
Contributor Author

devinbost commented Jan 21, 2020

I found an example involving a dead topic. (This was on a low-volume topic.)
image

@devinbost
Copy link
Contributor Author

Here's another case where we saw the topics on a particular broker freeze.
image
Notice that both topics on broker 09 froze.

@sijie
Copy link
Member

sijie commented Jan 22, 2020

Interestingly, the available permits seems to be fine (1000). Are you able to get some more stats that we can help debug?

@devinbost
Copy link
Contributor Author

We will need to capture some stats when this happens next. After my team made some changes to improve the stability of the Zookeeper cluster, the frequency of this issue decreased on v2.4.0. So, we will need to update one of the clusters to use 2.4.2 to reproduce this issue again.

@devinbost
Copy link
Contributor Author

I also noticed that some of the bookkeeper nodes are currently running:
apachepulsar/pulsar-all:2.3.0
and some of them are running:
streamlio/pulsar-all:2.4.0-streamlio-24

I don't know if that would have anything to do with the issue.

@sijie
Copy link
Member

sijie commented Jan 25, 2020

usually, this kind of problem is not related to bookkeeper. Is the broker running the same version and what is the version?

@devinbost
Copy link
Contributor Author

The brokers are all running streamlio/pulsar-all:2.4.0-streamlio-24

@devinbost
Copy link
Contributor Author

The brokers are also running as the function workers. (There aren't dedicated function worker instances.)

@sijie
Copy link
Member

sijie commented Jan 28, 2020

Given it is running a special version, it is hard for us to realize if there are any special changes in that version. I think it is better to provide some jstack or heap dump when this problem happens. Otherwise it is hard for the community to help here.

@devinbost
Copy link
Contributor Author

@sijieg Thanks for the advice.
I'll talk with the team about moving to a standard Pulsar release.
We did, however, experience this issue on the standard Pulsar 2.4.2 release.

@codelipenghui
Copy link
Contributor

@devinbost There is a fix #5894 for broker stop dispatch messages and it was released in 2.5.0.

@devinbost
Copy link
Contributor Author

devinbost commented Feb 11, 2020

This issue is a duplicate of #5311

@devinbost
Copy link
Contributor Author

devinbost commented Mar 10, 2020

We noticed that after one of these situations, our Zookeeper nodes had gotten out of sync. We ran a diff after scraping the Zookeeper data on each of the ZK instances, and we noticed that only one ZK instance was behind (although we weren't able to check the ledger data since it's constantly changing.) The difference we noticed was that the ZK instance that was behind was missing several nodes in: /admin/policies

@devinbost
Copy link
Contributor Author

@addisonj Have you noticed anything similar?

@devinbost
Copy link
Contributor Author

@lhotari I reproduced the issue with using this as the Pulsar Function:

public class RandomFilterFunction implements Function<String, String> {

    private Random rand = new Random();

    @Override
    public String process(String input, Context context) {
	    double passFraction = Double.parseDouble((String)context.getUserConfigValueOrDefault("pass-fraction", "0.5"));
	    if (rand.nextDouble() < passFraction) {
            return input;
        } else {
            return null;
	    }
    }
}

I chained one function (with parallelism of 4) to another function (with parallelism of 4), both using the same class.
The first function in the flow has a pass-fraction of "0.3", and the other one is default.

@devinbost
Copy link
Contributor Author

To throw a curveball, I was able to reproduce this issue where the subscription is completely frozen while using the function from the code shared above.
image
Interestingly, the consumer has negative permits on all four partitions, but the function isn't doing anything.
So, that leads me to believe (again) that pendingAcks have maxed out the semaphore, causing the function to block while it waits for ack's from the broker.

@devinbost
Copy link
Contributor Author

@rdhabalia This happened after I applied the default broker.conf settings (fixing the non-default configs for ManagedLedger and a few other things that we had) and restarted the brokers.

@devinbost
Copy link
Contributor Author

We discovered that certain non-default broker.conf settings seem to be key to reproducing this bug, but we haven't yet identified which of them are responsible.

    maxUnackedMessagesPerConsumer=0
    maxUnackedMessagesPerSubscription=0
    bookkeeperEnableStickyReads=true
    managedLedgerMaxEntriesPerLedger=10
    managedLedgerMinLedgerRolloverTimeMinutes=0
    managedLedgerReadEntryTimeoutSeconds=120
    managedLedgerAddEntryTimeoutSeconds=120

It's obvious that managedLedgerMaxEntriesPerLedger=10 puts significant load on the bookies and results in a high volume of rollovers, but interestingly, we weren't able to reproduce the freeze with only these configs:

    bookkeeperEnableStickyReads=true
    managedLedgerMaxEntriesPerLedger=10
    managedLedgerMinLedgerRolloverTimeMinutes=0
    managedLedgerReadEntryTimeoutSeconds=120
    managedLedgerAddEntryTimeoutSeconds=120

It was only after adding these that the bug re-appeared:

    maxUnackedMessagesPerConsumer=0
    maxUnackedMessagesPerSubscription=0

We also noticed an ERROR message on the affected broker that consistently appears (in Pulsar 2.6.2) at the exact time it freezes:

2021-05-25T21:11:05,208 [bookkeeper-ml-scheduler-OrderedScheduler-6-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger -1 in time-out 120 sec

While it's frozen, after waiting for a while, a series of similar errors will appear on that same broker but with an actual (not -1) ledgerId:

    2021-05-25T21:15:05,208 [bookkeeper-ml-scheduler-OrderedScheduler-6-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger 478895 in time-out 120 sec
    2021-05-25T21:17:05,208 [bookkeeper-ml-scheduler-OrderedScheduler-6-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger 478895 in time-out 120 sec
    2021-05-25T21:19:05,208 [bookkeeper-ml-scheduler-OrderedScheduler-6-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger 478895 in time-out 120 sec
    2021-05-25T21:21:05,208 [bookkeeper-ml-scheduler-OrderedScheduler-6-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger 478895 in time-out 120 sec

As soon as that broker is stopped, data will resume flowing briefly before freezing again on a different broker, which will show the error ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger -1 in time-out 120 sec when it freezes.

It looks like this issue is closely related to apache/bookkeeper#2716

We also noticed no clear JVM memory issues, though a few minutes after the freeze occurs, we did notice a memory leak that showed up after we added -Dio.netty.leakDetectionLevel=advanced -Dpulsar.allocator.leak_detection=Advanced arguments to PULSAR_MEM
At this point, it looks like the leak is a symptom, rather than a root cause. It also occurs after several of those ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Failed to add entry for ledger 478895 in time-out 120 sec logs have appeared on that broker. The leak message was ERROR io.netty.util.ResourceLeakDetector - LEAK: ByteBuf.release() was not called before it's garbage-collected.

The WARN-level logs around the leak are leak_log.txt.

@devinbost
Copy link
Contributor Author

devinbost commented May 26, 2021

Many thanks to @lhotari for figuring out how to reproduce this bug outside of my environment.

@lhotari
Copy link
Member

lhotari commented May 28, 2021

I have reported a separate issue about a direct memory leak, it's #10738 . It includes a full repro case with a helm deployment that can be used to reproduce the issue in minikube (requires sufficient RAM) (or any k8s environment).

@JohnMops
Copy link

Any news regarding this one?
We are seeing a topic stops every 9 seconds and needs to be reloaded.

@eolivelli
Copy link
Contributor

@JohnMops did you update to Pulsar 2.8.0?

@devinbost
Copy link
Contributor Author

devinbost commented Jun 20, 2021 via email

@devinbost
Copy link
Contributor Author

Related to #10813 except that issue seems to not occur if batching is disabled.

As an update, I've reproduced this bug even after correcting the non-standard broker.conf settings that were mentioned earlier in this issue.

@devinbost
Copy link
Contributor Author

devinbost commented Jul 3, 2021

I mapped out more of the ack flow, so I will add it to what I documented here and make it more readable.

The bracket notation below is intended to specify an instance of the class (to distinguish from a static method call.)

When ProducerImpl sends the messages, it builds newSend command instances,
which get picked up by ServerCnx.handleSend(..),
which goes to Producer.publishMessage(..),
then PersistentTopic.publishMessage(..)
which calls asyncAddEntry(headersAndPayload, publishContext),
which calls [PersistentTopic].ledger.asyncAddEntry(headersAndPayload, (int) publishContext.getNumberOfMessages(), this, publishContext)
which creates the OpAddEntry and calls internalAsyncAddEntry(addOperation) on a different thread,
which adds OpAddEntry to [ManagedLedgerImpl].pendingAddEntries

From somewhere (it's not clear to me exactly where yet) we call OpAddEntry.safeRun(),
which polls pendingAddEntries,
gets the callback on OpAddEntry (which is the PersistentTopic instance)
and calls [PersistentTopic].addComplete(lastEntry, data.asReadOnly(), ctx),
which calls publishContext.completed(..) on Producer.MessagePublishContext,
which calls Producer.ServerCnx.execute(this) on the MessagePublishContext,
which calls MessagePublishContext.run(),
which triggers Producer.ServerCnx.getCommandSender().sendSendReceiptResponse(..) [SEND_RECEIPT],
which writes a newSendReceiptCommand to the channel.

From there, the client gets the SEND_RECEIPT command from PulsarDecoder,
which calls [ClientCnx].handleSendReceipt(..),
which calls [ProducerImpl].ackReceived(..),
which calls releaseSemaphoreForSendOp(..) to release the semaphore.
The semaphore doesn't block until maxPendingMessages is reached, which is 1000 by default.

After a rollover, pendingAddEntries is also polled. (Could this access race with the other path that's polling pendingAddEntries?) Here's how this happens:

Something triggers [ManagedLedgerImpl].createComplete(..),
which calls [ManagedLedgerImpl].updateLedgersListAfterRollover(MetaStoreCallback<Void>),
which calls [MetaStoreImpl].asyncUpdateLedgerIds(..),
which calls callback.operationComplete(..) on the MetaStoreCallback<Void>,
which calls [ManagedLedgerImpl].updateLedgersIdsComplete(..),
which polls for pendingAddEntries
which calls op.initiate() on each OpAddEntry in pendingAddEntries,
which calls ledger.asyncAddEntry(..),
which calls [LedgerHandle].asyncAddEntry(..),
which calls [LedgerHandle].doAsyncAddEntry(op),
which adds the op to pendingAddOps on the LedgerHandle instance,
which calls [PendingAddOp].cb.addCompleteWithLatency(..)
which calls the callback on [PendingAddOp], which is [OpAddEntry]
which calls addCompleteWithLatency on the AddCallback interface,
which calls [OpAddEntry].addComplete(..)
which, if it succeeded:
changes its state and recycles it if completed or if there was a failure, it triggers [OpAddEntry].safeRun()
which removes itself from [ManagedLedgerImpl].pendingAddEntries
and triggers cb.addComplete(..) on [PersistentTopic],
which triggers publishContext.completed(..) on [Producer.MessagePublishContext]
which triggers producer.publishOperationCompleted()
which decrements pendingPublishAcks

It's still not clear where the ack's are getting lost, so there must be another part of the flow that I'm missing.

@devinbost
Copy link
Contributor Author

devinbost commented Jul 6, 2021

After spending many hours digging through the ack paths and not finding any issues, I took another look at the client (ProducerImpl), and I noticed something interesting in the heap dump.
ProducerImpl is in a Connecting state, so it's waiting to get the connection.
However, [ProducerImpl].connectedSince says it's been connected for about 3 days...
If it's in a Connecting state, that implies it was disconnected at some point.

If there's a connectivity issue, that would explain why this bug has been so hard to reproduce and why it can't be reproduced locally. It could be that some network hardware is doing something weird with the connection, and the client doesn't handle it correctly and gets stuck in a Connecting state and doesn't try re-establishing the connection. So, it's just waiting forever for the connection to establish.
When in the Connecting state, in ProducerImpl.sendAsync(..) , ProducerImpl.isValidProducerState(..) returns true even though we haven't completed the establishment of the connection. When that method returns true, it allows the producer to enqueue messages.
Sure enough, [ProducerImpl].connectionHandler.state.pendingMessages contains all (exactly) 1000 of the OpSendMsg that are blocking the semaphore. (The semaphore blocks once 1000 messages accumulate that haven't been ack'd.)
It also has exactly 1000 null value entries for pendingCallbacks, but I'm not sure if that means anything.

In the function logs, I'm noticing a lot of messages like this:

00:30:00.452 [pulsar-timer-5-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [persistent://myTenant/processing/random-1-out] [green-test-126-142] Reconnecting after connection was closed
00:30:00.453 [pulsar-client-io-1-2] INFO  org.apache.pulsar.client.impl.ProducerImpl - [persistent://myTenant/processing/random-1-out] [green-test-126-142] Creating producer on cnx [id: 0xd7dc2240, L:/10.20.55.42:45836 - R:server10.example.com.com/10.20.55.42:6650]
00:30:59.028 [pulsar-timer-5-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [persistent://myTenant/processing/random-1-out] [green-test-126-142] Reconnecting after connection was closed
00:30:59.029 [pulsar-client-io-1-2] INFO  org.apache.pulsar.client.impl.ProducerImpl - [persistent://myTenant/processing/random-1-out] [green-test-126-142] Creating producer on cnx [id: 0xd7dc2240, L:/10.20.55.42:45836 - R:server10.example.com.com/10.20.55.42:6650]

It looks like the function keeps trying to reconnect, but the connection is immediately (within 1 millisecond) closed. So, the function is never able to re-establish a healthy connection. If it can't establish a healthy connection, it can't produce the messages or receive ack's from the broker. So, this is a viable root cause.

@devinbost
Copy link
Contributor Author

This bug has been resolved in DataStax Luna Streaming 2.7.2_1.1.21

@skyrocknroll
Copy link
Contributor

@devinbost Which version of apache pulsar has this fix ?

@marcioapm
Copy link

This bug has been resolved in DataStax Luna Streaming 2.7.2_1.1.21

Hi @devinbost is this is 2.9.1?

@lhotari
Copy link
Member

lhotari commented Jan 13, 2022

Which version of apache pulsar has this fix ?

@skyrocknroll @marcioapm All required fixes are included in Apache Pulsar 2.7.4 and 2.8.2 versions.

@lhotari lhotari closed this as completed Jan 13, 2022
@lhotari
Copy link
Member

lhotari commented Apr 12, 2022

#15031 and #15067 resolve an issue with a similar symptom, where subscriptions freeze.

@JohnMops
Copy link

Still have stalled topic issue in 2.9.2

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.