You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kudu.apache.org by "Adar Dembo (JIRA)" <ji...@apache.org> on 2017/04/13 02:42:42 UTC
[jira] [Commented] (KUDU-1913) Tablet server runs out of threads
when creating lots of tablets
[ https://issues.apache.org/jira/browse/KUDU-1913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15967031#comment-15967031 ]
Adar Dembo commented on KUDU-1913:
----------------------------------
A couple notes about this:
We currently rely on there being a single prepare thread per tablet in order to serialize writes via Raft replication. If these threads were aggregated across the tserver, we'd want a way to ensure that writes from the same tablet are processed serially. Chromium's [Sequenced Worker Pool|https://cs.chromium.org/chromium/src/base/threading/sequenced_worker_pool.h?q=base::SequencedWorkerPool&sq=package:chromium&l=72&type=cs] might be a good fit for this.
[MultiRaft|https://www.cockroachlabs.com/blog/scaling-raft/] is an approach adopted by CockroachDB to improve Raft scalability when a server has many tablets. It could be worth exploring for our purposes too, though I see CockroachDB is [now using etcd's Raft implementation|https://github.com/cockroachdb/cockroach/issues/20]; I don't know if it implements MultiRaft or not.
> Tablet server runs out of threads when creating lots of tablets
> ---------------------------------------------------------------
>
> Key: KUDU-1913
> URL: https://issues.apache.org/jira/browse/KUDU-1913
> Project: Kudu
> Issue Type: Sub-task
> Components: consensus, log
> Reporter: Juan Yu
> Labels: data-scalability
>
> When adding lots of range partitions, all tablet server crashed with the following error:
> F0308 14:51:04.109369 12952 raft_consensus.cc:1985] Check failed: _s.ok() Bad status: Runtime error: Could not create thread: Resource temporarily unavailable (error 11)
> Tablet server should handle error/failure more gracefully instead of crashing.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)