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

[Broker] Increase default numHttpServerThreads value to 50 to prevent Admin API unavailability #14320

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

lhotari
Copy link
Member

@lhotari lhotari commented Feb 16, 2022

Motivation

Since Pulsar Admin API uses the blocking servlet API, it is possible that all Jetty threads are occupied and this causes unavailability on the Pulsar Admin API. The default value for the maximum number of threads for Jetty is too low in Pulsar. That is the root cause of many problems where Pulsar Admin API is unavailable when all threads are in use.

Additional context

Mailing list thread about "make async" changes: https://lists.apache.org/thread/tn7rt59cd1k724l4ytfcmzx1w2sbtw7l

Modification

  • Jetty defaults to 200 maximum threads, to prevent thread pool starvation. Make Pulsar use the value of 50 maximum threads by setting numHttpServerThreads=50.
  • Update the documentation for numHttpServerThreads

…t Broker Admin API unavailability

- numHttpServerThreads is the maximum number of threads.
  Initial number of threads is 8 when numHttpServerThreads >= 8

- Jetty defaults to 200 maximum threads, to prevent thread pool starvation.
  - this is the reason to use the value 200 as a default
@lhotari lhotari added type/bug The PR fixed a bug or issue reported a bug area/broker doc-not-needed Your PR changes do not impact docs labels Feb 16, 2022
@lhotari lhotari self-assigned this Feb 16, 2022
eolivelli
eolivelli previously approved these changes Feb 16, 2022
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch

@gaoran10 @michaeljmarshall please consider including this fix in your upcoming RCs

nicoloboschi
nicoloboschi previously approved these changes Feb 16, 2022
Copy link
Contributor

@nicoloboschi nicoloboschi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change the default configuration need to start with a proposal

Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the reason of setting the default value to 200? If the node just have one core, what will happen? Please send email to dev mail list to discuss.

@lhotari
Copy link
Member Author

lhotari commented Feb 16, 2022

What's the reason of setting the default value to 200? If the node just have one core, what will happen? Please send email to dev mail list to discuss.

These are threads. Jetty defaults to 200 maximum threads, to prevent thread pool starvation. This is recommended when using blocking Servlet API. The problem is that Pulsar uses the blocking servlet API and doesn't have a sufficient amount of threads which are needed and recommended.

The value 200 doesn't mean that there will be 200 threads to start with. This is the maximum size for the thread pool. When the value is more than 8, Jetty will start with 8 initial threads and add more threads to the pool when all threads are occupied.

I have already started an email discussion to discuss this topic. Please reply to https://lists.apache.org/thread/byg1g081o6mfj0xn8ntryvb5qplmrjyl .

There is useful background information in https://lists.apache.org/thread/hso8qwsv40ccrk116fj5ggdpt3b9d4g4 . I wrote that reply before I noticed Penghui's response. It contains a link to Jetty's documenation about asynchronous servlets: https://wiki.eclipse.org/Jetty/Feature/Continuations#Why_Asynchronous_Servlets_.3F .

@Jason918
Copy link
Contributor

These are threads. Jetty defaults to 200 maximum threads,

@lhotari If this is jetty defaults. Can we just leave it blank?

@lhotari
Copy link
Member Author

lhotari commented Feb 16, 2022

Change the default configuration need to start with a proposal

This PR is a proposal. I have also made this proposal on the dev mailing list in the discussion. https://lists.apache.org/thread/byg1g081o6mfj0xn8ntryvb5qplmrjyl . What else is needed?

In this case, the previous default for numHttpServerThreads is simply too small and invalid when blocking servlet API is used.
The value 200 doesn't mean that there will be 200 threads to start with. This is the maximum size for the thread pool. When the value is more than 8, Jetty will start with 8 initial threads and add more threads to the pool when all threads are occupied.

There is no breaking change in increasing the default value to 200. It's just an improvement and fixes "the problem" where Admin API goes unresponsive when all threads are occupied.

We might end up setting the default value to something lower than 200. A value like 50 or 100 might be fine. I just think that 200 is a good default since Jetty also uses that as the default value.

The main overhead of a thread is the amount of memory that the stack of each thread consumes. It's 1MB by default. 200 threads will consume 200MB of RSS memory in the thread stacks.

@lhotari
Copy link
Member Author

lhotari commented Feb 16, 2022

@lhotari If this is jetty defaults. Can we just leave it blank?

@Jason918 no. Pulsar overrides the default with the value set in numHttpServerThreads in the configuration.

Pulsar code locations:

this.webServiceExecutor = new WebExecutorThreadPool(
pulsar.getConfiguration().getNumHttpServerThreads(),
"pulsar-web");

public WebExecutorThreadPool(int maxThreads, String namePrefix) {
super(maxThreads);
this.threadFactory = new DefaultThreadFactory(namePrefix);
}

Copy link
Member

@RobertIndie RobertIndie left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't seem to solve the root cause of the issue.

This PR is a proposal. I have also made this proposal on the dev mailing list in the discussion. https://lists.apache.org/thread/byg1g081o6mfj0xn8ntryvb5qplmrjyl . What else is needed?

I think we need a PIP to get this change approved.

@hangc0276
Copy link
Contributor

These are threads. Jetty defaults to 200 maximum threads

@lhotari Do we need to take the number of availableProcessors into consideration for the maximum threads of the thread pool?

@lhotari
Copy link
Member Author

lhotari commented Feb 16, 2022

This doesn't seem to solve the root cause of the issue.

Please tell me what is "the issue" that you are referring to?

This PR is a proposal. I have also made this proposal on the dev mailing list in the discussion. https://lists.apache.org/thread/byg1g081o6mfj0xn8ntryvb5qplmrjyl . What else is needed?

I think we need a PIP to get this change approved.

The Pulsar community makes major decisions on the dev mailing list according to the Apache Way. The mailing list is the place to decide whether this change needs a PIP or not. Please participate in the existing dev mailing list discussion: https://lists.apache.org/thread/byg1g081o6mfj0xn8ntryvb5qplmrjyl

@shoothzj
Copy link
Member

@lhotari @eolivelli I suggest that we should consider the system cpu cores, it may be hurtful change for people who run pulsar in a low machine, like one cpu core.

@merlimat
Copy link
Contributor

We shouldn't add some temporary PRs during the release phase unless they are really important.

@BewareMyPower This PR contains a very important change. The recommended maximum thread count for Jetty thread pool is 50-500. Pulsar current uses an invalid valid.

@lhotari This is not a new thing and it's something that can easily be changed by users. I don't think we need to rush into a release.

The sync->async changes in the Pulsar Admin API are not needed when a proper value is used.

The problem is not just keeping threads busy but the cases in which you have calls that spawn to multiple brokers. In this cases, the HTTP call can come back to same broker and we can have a deadlock, no matters how many threads we have in the pool.

@codelipenghui
Copy link
Contributor

@codelipenghui #14329 has been published. Please follow up on the mailing list.

Sure

@BewareMyPower
Copy link
Contributor

BewareMyPower commented Feb 17, 2022

We shouldn't add some temporary PRs during the release phase unless they are really important.

@BewareMyPower This PR contains a very important change. The recommended maximum thread count for Jetty thread pool is 50-500. Pulsar current uses an invalid valid. The sync->async changes in the Pulsar Admin API are not needed when a proper value is used.

I think I need to explain more for the important term I used. IMO, a PR that could block a release during the release phase must match following rules:

  1. It must be a bug fix.
  2. The bug was introduced from the current release, i.e. it's a regression.
  3. There is no workaround.

It's only my opinion. I think our release document for release manager missed something like this.

Let's look back to this PR. First, I don't think a change to the default configuration value can be treated as a bug fix. It's more like an enhancement. Because the previous stable releases all should have the same problem. Then, we can see it's not a regression. Third, it's not something serious like Log4j2 Vulnerability (CVE-2021-44228). It just make some certain cases not work for Admin API and can be fixed by configuration tuning.

In short, IMO, after a release started, we must be very careful and strict on the new PRs.

@nodece
Copy link
Member

nodece commented Feb 17, 2022

The unavailability of the Admin API is not caused by the HTTP server thread, the root cause is that the ZK callback thread is blocked.

When an admin API calls the ZK metadatastore API, it gets the ZK data by call the CompletableFuture, note that we did not use the executor to execute the CompletableFuture#complete() in ZKMetadataStore.java#L171. In ZK callback thread, once the caller converts async to sync calls then the ZK callback thread will be blocked, this code so like: metadata.getAsync().get(30, TimeUnit.SECONDS).

How to solve this problem?

  1. Use an executor to execute the callback that passes data to Pulsar in ZK callback
  2. Don't convert async to sync calls, so there are make some PR that converts sync to async calls

How to reproduce the ZK callback thread is blocked:

docker run -d -p 2181:2181 --name test-zookeeper zookeeper
public class Main {
    private static final long CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5);

    public static void printThread(String name) {
        System.out.println(name + " thread name -> " + Thread.currentThread().getName());
    }

    public static void main(String[] args) throws Exception {
        ZooKeeper zkc = new ZooKeeper("localhost:2181", 60_000, null);

        System.out.println("Check the zk connect");
        CountDownLatch zkLatch = new CountDownLatch(1);
        new Thread(() -> {
            while (true) {
                if (zkc.getState().isConnected()) {
                    zkLatch.countDown();
                    break;
                }
            }
        }).start();
        if (!zkLatch.await(5, TimeUnit.SECONDS)) {
            throw new Exception("zk connect failed");
        }

        AsyncLoadingCache<String, byte[]> objCache = Caffeine.newBuilder()
                .refreshAfterWrite(CACHE_REFRESH_TIME_MILLIS, TimeUnit.MILLISECONDS)
                .buildAsync((key, executor) -> {
                    CompletableFuture<byte[]> future = new CompletableFuture<>();
                    zkc.multi(Lists.newArrayList(Op.getData("/")), (rc, path, ctx, opResults) -> {
                        printThread("zk callback");
                        future.complete(null);
                    }, null);
                    return future;
                });

        CountDownLatch countDownLatch = new CountDownLatch(1);

        // Reproduce the ZK callback is blocked
        System.out.println("async get start");
        objCache.get("/").whenComplete((unused, ignored) -> {
            printThread("async get done");
            try {
                System.out.println("zk thread will blocked after sync get");
                System.out.println("sync get start");
                objCache.get("/1").get(5, TimeUnit.SECONDS);
                // Unreachable
                printThread("sync get done");
                countDownLatch.countDown();
            } catch (Exception e) {
                e.printStackTrace();
            } finally {
                countDownLatch.countDown();
            }
        });

        countDownLatch.await();
    }
}

@lhotari
Copy link
Member Author

lhotari commented Feb 17, 2022

When an admin API calls the ZK metadatastore API, it gets the ZK data by call the CompletableFuture, note that we did not use the executor to execute the CompletableFuture#complete() in ZKMetadataStore.java#L171. In ZK callback thread, once the caller converts async to sync calls then the ZK callback thread will be blocked, this code so like: metadata.getAsync().get(30, TimeUnit.SECONDS).

The blocked thread in #13666 is a HTTP server thread.

"pulsar-web-40-28" #238 prio=5 os_prio=0 tid=0x00007f5a4000d800 nid=0x2bcf waiting on condition [0x00007f5961d3b000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000005c5529d40> (a java.util.concurrent.CompletableFuture$Signaller)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
	at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.internalDeleteSubscriptionForNonPartitionedTopic(PersistentTopicsBase.java:1498)

I'll clarify what I have been referring to as "sync -> async" changes: changes where the use of the blocking Servlet API is migrated to use Asynchronous Servlet API.
I understand that it's necessary to not block in Zookeeper callbacks, but that is a different problem, which isn't related to Servlet API change.

@lhotari
Copy link
Member Author

lhotari commented Feb 17, 2022

@BewareMyPower This PR contains a very important change. The recommended maximum thread count for Jetty thread pool is 50-500. Pulsar current uses an invalid valid.

@lhotari This is not a new thing and it's something that can easily be changed by users. I don't think we need to rush into a release.

I agree. Have I been rushing this? Instead, you can say that a lot of PRs with "make async" have been pushed and merged recently.
This has introduced several known regressions that have been fixed. We don't know which are regressions that just haven't been found yet.

My valid questions were never answered by the contributors of the "make async" changes.

image

I'm expecting that there are issues or a PIP which is referred to.
@merlimat WDYT?

The sync->async changes in the Pulsar Admin API are not needed when a proper value is used.

The problem is not just keeping threads busy but the cases in which you have calls that spawn to multiple brokers. In this cases, the HTTP call can come back to same broker and we can have a deadlock, no matters how many threads we have in the pool.

I'll clarify: what I have been referring to as "sync -> async" changes: changes where the use of the blocking Servlet API is migrated to use Asynchronous Servlet API. That won't solve any problems on it's own. Any problems that it might solve would be solved also by configuring Jetty as it is recommended to be configured when there are blocking calls involved.
The recommended maximum thread pool size is 50 to 500 for Jetty. I have been going through some details and there are multiple other things that are not properly configured. I'll be following up with separate PRs. (UPDATE: the draft PR which fixes backpressure handling is #14353).

I'd assume that the reason for deadlocks when thread pool size is properly configured are caused by locks. I like to see an example of a deadlock which couldn't be resolved by continuing to use the blocking servlet api. I'm not against the changes from blocking API to async API, but I think changes need proper justification, especially when the "make async" changes have been initiated without referring any reported issues or a PIP.

@lhotari
Copy link
Member Author

lhotari commented Feb 17, 2022

I think I need to explain more for the important term I used. IMO, a PR that could block a release during the release phase must match following rules:

I haven't requested to block the release.

Let's look back to this PR. First, I don't think a change to the default configuration value can be treated as a bug fix. It's more like an enhancement. Because the previous stable releases all should have the same problem. Then, we can see it's not a

The Jetty documentation recommends values 50-500 for the maximum thread pool size. That is a fact, so there cannot be different opinions on this. Since the default configuration value doesn't fall in the recommended value range, my opinion is that this is a bug. For some people a bug is a feature. :) Does it really matter whether we call this a bug or an improvement?
The fact is that the current value doesn't fall in the recommended value range.

@lhotari lhotari dismissed stale reviews from nicoloboschi and eolivelli via ee8e67f February 17, 2022 10:29
@BewareMyPower
Copy link
Contributor

Instead, you can say that a lot of PRs with "make async" have been pushed and merged recently.

Yeah, I noticed these PRs recently as well. But are these PRs blockers for 2.10.0 release? IMO, they should not be blockers as well. I've thought they are intended to be included in Pulsar 2.11.0.

I haven't requested to block the release.

Sorry, I might missed some context. I just saw this PR in the 2.10.0 release email list. This PR should focus on the fix itself, but the previous discussion might go far for the release issue.

My valid questions were never answered by the contributors of the "make async" changes.

It's a pity to see the lack of communication. AFAIK, @Technoboy- is also preparing for a PIP to make admin APIs async. I think you should have a discussion about:

  1. Whether this PR solve the root problem?
  2. Based on this PR, is making admin APIs async meaningful?

@BewareMyPower
Copy link
Contributor

I'll continue the discussion in the PIP-142 discussion email.

@lhotari
Copy link
Member Author

lhotari commented Feb 17, 2022

My valid questions were never answered by the contributors of the "make async" changes.

It's a pity to see the lack of communication. AFAIK, @Technoboy- is also preparing for a PIP to make admin APIs async. I think you should have a discussion about:

  1. Whether this PR solve the root problem?

Great point @BewareMyPower . I hope that the problem would first be discussed or reported before a PIP is created. @Technoboy- Would you be able to start some discussion even before the PIP is ready?

  1. Based on this PR, is making admin APIs async meaningful?

That's also a valid question to ask. When we work together, we can learn together.

@eolivelli
Copy link
Contributor

This is not something that can block release.
The value is already configurable.
I believe that there is no hurry in committing this change, and we can discuss about a new value or decide that the default should not be changed.

@nodece
Copy link
Member

nodece commented Feb 17, 2022

When an admin API calls the ZK metadatastore API, it gets the ZK data by call the CompletableFuture, note that we did not use the executor to execute the CompletableFuture#complete() in ZKMetadataStore.java#L171. In ZK callback thread, once the caller converts async to sync calls then the ZK callback thread will be blocked, this code so like: metadata.getAsync().get(30, TimeUnit.SECONDS).

The blocked thread in #13666 is a HTTP server thread.

"pulsar-web-40-28" #238 prio=5 os_prio=0 tid=0x00007f5a4000d800 nid=0x2bcf waiting on condition [0x00007f5961d3b000]
   java.lang.Thread.State: WAITING (parking)
	at sun.misc.Unsafe.park(Native Method)
	- parking to wait for  <0x00000005c5529d40> (a java.util.concurrent.CompletableFuture$Signaller)
	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
	at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.internalDeleteSubscriptionForNonPartitionedTopic(PersistentTopicsBase.java:1498)

I'll clarify what I have been referring to as "sync -> async" changes: changes where the use of the blocking Servlet API is migrated to use Asynchronous Servlet API. I understand that it's necessary to not block in Zookeeper callbacks, but that is a different problem, which isn't related to Servlet API change.

When the ZK callback thread is blocked in the WEB thread, another admin API request the ZK metadata store is not working, so you see this thread stack.

@github-actions
Copy link

The pr had no activity for 30 days, mark with Stale label.

@lhotari lhotari changed the title [Broker] Increase default numHttpServerThreads value to 200 to prevent Admin API unavailability [Broker] Increase default numHttpServerThreads value to 50 to prevent Admin API unavailability Apr 8, 2022
@github-actions
Copy link

The pr had no activity for 30 days, mark with Stale label.

@github-actions github-actions bot added the Stale label May 28, 2022
@dave2wave
Copy link
Member

@lhotari - should this old debate on a PR be closed as it was hopefully resolved?

@Technoboy- Technoboy- added this to the 3.2.0 milestone Jul 31, 2023
@Technoboy- Technoboy- modified the milestones: 3.2.0, 3.3.0 Dec 22, 2023
@lhotari
Copy link
Member Author

lhotari commented Feb 12, 2024

The Pulsar Admin client doesn't have a limit of how many connections it opens to a single broker. There is issue #22041 for addressing that.

@dao-jun
Copy link
Member

dao-jun commented Feb 18, 2024

I support increase the value of numHttpServerThreads, but it seems we will create a FixedThreadPoolExecutor with 200 threads?
image
image

by the way, do we still working on this PR? The PR has been blocked for 2 yrs.

@coderzc coderzc modified the milestones: 3.3.0, 3.4.0 May 8, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker doc-not-needed Your PR changes do not impact docs lifecycle/stale Stale type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet