You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Patricio Echague (JIRA)" <ji...@apache.org> on 2013/10/09 02:29:42 UTC
[jira] [Created] (CASSANDRA-6169) Too many splits causes a
"OutOfMemoryError: unable to create new native thread" in
AbstractColumnFamilyInputFormat
Patricio Echague created CASSANDRA-6169:
-------------------------------------------
Summary: Too many splits causes a "OutOfMemoryError: unable to create new native thread" in AbstractColumnFamilyInputFormat
Key: CASSANDRA-6169
URL: https://issues.apache.org/jira/browse/CASSANDRA-6169
Project: Cassandra
Issue Type: Bug
Environment: 1.2.10
vnodes (server side)
Mac OS x (client)
Reporter: Patricio Echague
Priority: Minor
The problem is caused by having 2300+ tokens due to vnodes.
In the client side I get this exception
{code}
Exception in thread "main" java.lang.OutOfMemoryError: unable to create new native thread
at java.lang.Thread.start0(Native Method)
at java.lang.Thread.start(Thread.java:691)
at java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:943)
at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1336)
at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:132)
at org.apache.cassandra.hadoop.AbstractColumnFamilyInputFormat.getSplits(AbstractColumnFamilyInputFormat.java:187)
at org.apache.hadoop.mapred.JobClient.writeNewSplits(JobClient.java:1054)
at org.apache.hadoop.mapred.JobClient.writeSplits(JobClient.java:1071)
at org.apache.hadoop.mapred.JobClient.access$700(JobClient.java:179)
at org.apache.hadoop.mapred.JobClient$2.run(JobClient.java:983)
at org.apache.hadoop.mapred.JobClient$2.run(JobClient.java:936)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1190)
at org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:936)
at org.apache.hadoop.mapreduce.Job.submit(Job.java:550)
at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:580)
at com.relateiq.hadoop.cassandra.etl.CassandraETLJob.run(CassandraETLJob.java:58)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
at com.relateiq.hadoop.cassandra.etl.CassandraETLJob.main(CassandraETLJob.java:149)
{code}
The problem seem to be in AbstractColumnFamilyInputFormat line ~180 which has an unbounded upper limit (actually it is Integer.MAX_INT)
{code}
ExecutorService executor = Executors.newCachedThreadPool();
{code}
Followed by:
{code}
for (TokenRange range : masterRangeNodes)
{
if (jobRange == null)
{
// for each range, pick a live owner and ask it to compute bite-sized splits
splitfutures.add(executor.submit(new SplitCallable(range, conf)));
}
else
.....
{code}
which gets called one time per token and creates one thread just as many times.
The easy fix unless there is a longer term fix I'm unaware of would be to set an upper limit to the thread pool.
Something like this:
{code}
ExecutorService executor = new ThreadPoolExecutor(0, ConfigHelper.getMaxConcurrentSplitsResolution(), 60L, TimeUnit.SECONDS, new SynchronousQueue<Runnable>());
{code}
Shall I proceed with a patch ?
--
This message was sent by Atlassian JIRA
(v6.1#6144)