Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E2319970B for ; Thu, 29 Sep 2011 17:03:39 +0000 (UTC) Received: (qmail 82580 invoked by uid 500); 29 Sep 2011 17:03:39 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 82557 invoked by uid 500); 29 Sep 2011 17:03:39 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 82548 invoked by uid 99); 29 Sep 2011 17:03:39 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Sep 2011 17:03:39 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Sep 2011 17:03:36 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 5F5F62388847 for ; Thu, 29 Sep 2011 17:03:15 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1177365 - in /cassandra/branches/cassandra-1.0.0: CHANGES.txt src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java Date: Thu, 29 Sep 2011 17:03:15 -0000 To: commits@cassandra.apache.org From: jbellis@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20110929170315.5F5F62388847@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: jbellis Date: Thu Sep 29 17:03:14 2011 New Revision: 1177365 URL: http://svn.apache.org/viewvc?rev=1177365&view=rev Log: fix full queue scenario for ParallelCompactionIterator patch by jbellis; reviewed by slebresne for CASSANDRA-3270 Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/CHANGES.txt?rev=1177365&r1=1177364&r2=1177365&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0.0/CHANGES.txt (original) +++ cassandra/branches/cassandra-1.0.0/CHANGES.txt Thu Sep 29 17:03:14 2011 @@ -9,6 +9,7 @@ * Keep SimpleSnitch proximity ordering unchanged from what the Strategy generates, as intended (CASSANDRA-3262) * fix counter entry in jdbc TypesMap (CASSANDRA-3268) + * fix full queue scenario for ParallelCompactionIterator (CASSANDRA-3270) 1.0.0-rc1 Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java?rev=1177365&r1=1177364&r2=1177365&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java (original) +++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/db/compaction/ParallelCompactionIterable.java Thu Sep 29 17:03:14 2011 @@ -144,21 +144,13 @@ public class ParallelCompactionIterable private class Reducer extends MergeIterator.Reducer { private final List rows = new ArrayList(); - - private final ThreadPoolExecutor executor; private int row = 0; - private Reducer() - { - super(); - executor = new ThreadPoolExecutor(Runtime.getRuntime().availableProcessors(), - Runtime.getRuntime().availableProcessors(), - Integer.MAX_VALUE, - TimeUnit.MILLISECONDS, - new SynchronousQueue(), - new NamedThreadFactory("CompactionReducer")); - executor.setRejectedExecutionHandler(DebuggableThreadPoolExecutor.blockingExecutionHandler); - } + private final ThreadPoolExecutor executor = new DebuggableThreadPoolExecutor(Runtime.getRuntime().availableProcessors(), + Integer.MAX_VALUE, + TimeUnit.MILLISECONDS, + new SynchronousQueue(), + new NamedThreadFactory("CompactionReducer")); public void reduce(RowContainer current) {