Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 090FD66BE for ; Thu, 26 May 2011 16:05:51 +0000 (UTC) Received: (qmail 21874 invoked by uid 500); 26 May 2011 16:05:45 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 21821 invoked by uid 500); 26 May 2011 16:05:45 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 21775 invoked by uid 99); 26 May 2011 16:05:45 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 May 2011 16:05:45 +0000 X-ASF-Spam-Status: No, hits=-0.7 required=5.0 tests=FREEMAIL_FROM,RCVD_IN_DNSWL_LOW,RFC_ABUSE_POST,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of jbellis@gmail.com designates 74.125.82.44 as permitted sender) Received: from [74.125.82.44] (HELO mail-ww0-f44.google.com) (74.125.82.44) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 May 2011 16:05:40 +0000 Received: by wwa36 with SMTP id 36so774509wwa.25 for ; Thu, 26 May 2011 09:05:19 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:mime-version:in-reply-to:references:from:date :message-id:subject:to:content-type; bh=LaHZHODuUVWx8DZ05rm+K3dr1GzhVPnXtpLLqOK6G9w=; b=rLM617EpRbKovl3JV9+DnZHVp4g51r+alBegN3YsZCUKNzbGO+YX8C+F0A+bJneePt xRV+Gmol4rU54MRfnvBraxFkEGEg3OqaRqnN7Oo8Wq7gfhve16yk+AihdztnmGOBXceJ 5lEzlePKIdgFJRVYhnUVYps6wTiDbCdglljrM= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:from:date:message-id:subject:to :content-type; b=n1t8T1otE1yIKE5WnHPx1KI6dFNnXgZ1xx1elJpnlQOHJkIsTCuelILrFDOio8Gvk0 tMl8GaRXKdLjO62M0QmSslweu2nwx+HaI1RxHlSe10NQfhicbOLXleV3F3RCIqsqa4ve vxBQOdMDuDeubQPGb1lhjemEAe0a+SaAcMG1Q= Received: by 10.216.254.37 with SMTP id g37mr1010954wes.36.1306425919139; Thu, 26 May 2011 09:05:19 -0700 (PDT) MIME-Version: 1.0 Received: by 10.216.17.15 with HTTP; Thu, 26 May 2011 09:04:59 -0700 (PDT) In-Reply-To: <4DDE6368.5090508@list-group.com> References: <4DDE6368.5090508@list-group.com> From: Jonathan Ellis Date: Thu, 26 May 2011 11:04:59 -0500 Message-ID: Subject: Re: OOM recovering failed node with many CFs To: user@cassandra.apache.org Content-Type: text/plain; charset=ISO-8859-1 Sounds like a legitimate bug, although looking through the code I'm not sure what would cause a tight retry loop on migration announce/rectify. Can you create a ticket at https://issues.apache.org/jira/browse/CASSANDRA ? As a workaround, I would try manually copying the Migrations and Schema sstable files from the system keyspace of the live node, then restart the recovering one. On Thu, May 26, 2011 at 9:27 AM, Flavio Baronti wrote: > I can't seem to be able to recover a failed node on a database where i did > many updates to the schema. > > I have a small cluster with 2 nodes, around 1000 CF (I know it's a lot, but > it can't be changed right now), and ReplicationFactor=2. > I shut down a node and cleaned its data entirely, then tried to bring it > back up. The node starts fetching schema updates from the live node, but the > operation fails halfway with an OOME. > After some investigation, what I found is that: > > - I have a lot of schema updates (there are 2067 rows in the system.Schema > CF). > - The live node loads migrations 1-1000, and sends them to the recovering > node (Migration.getLocalMigrations()) > - Soon afterwards, the live node checks the schema version on the recovering > node and finds it has moved by a little - say it has applied the first 3 > migrations. It then loads migrations 3-1003, and sends them to the node. > - This process is repeated very quickly (sends migrations 6-1006, 9-1009, > etc). > > Analyzing the memory dump and the logs, it looks like each of these 1000 > migration blocks are composed in a single message and sent to the > OutboundTcpConnection queue. However, since the schema is big, the messages > occupy a lot of space, and are built faster than the connection can send > them. Therefore, they accumulate in OutboundTcpConnection.queue, until > memory is completely filled. > > Any suggestions? Can I change something to make this work, apart from > reducing the number of CFs? > > Flavio > -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of DataStax, the source for professional Cassandra support http://www.datastax.com