I'm trying to get all the questions answered at once here so people aren't seeing a ton of emails from me.
>If you stop doing client requests does that number start to go down? I stopped the client at 11:28. There were 2306 files in data/Keyspace1. It's now 12:44, and there are 1826 files in data/Keyspace1. As I wrote this email, the number increased to 1903, then to 1938 and 2015, even though the server has no clients. I used jconsole to invoke a few explicit garbage collections and the number went down to 811. jconsole reports that the compaction pool has 1670 pending tasks. As I wrote this email, the number gradually increased to 1673. The server has no clients, so this is odd. The number of completed tasks in the compaction pool has consistently been going up while the number of pending tasks stays the same. The number of completed tasks increased from 130 to 136. > If not there is a bug. If there are exceptions in the logs then it is > an obvious one otherwise something more subtle. The output from "grep -v INFO log*" is reasonably concise: log.2009-12-02-19: WARN [Timer-0] 2009-12-02 19:55:23,305 LoadDisseminator.java (line 44) Exception was generated at : 12/02/2009 19:55:22 on thread Timer-0 log.2009-12-02-19: log.2009-12-02-19:java.lang.NullPointerException log.2009-12-02-19: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:198) log.2009-12-02-19: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:200) log.2009-12-02-19: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:200) log.2009-12-02-19: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpace(FileUtils.java:211) log.2009-12-02-19: at org.apache.cassandra.service.StorageService.getLoad(StorageService.java:510) log.2009-12-02-19: at org.apache.cassandra.service.LoadDisseminator.run(LoadDisseminator.java:37) log.2009-12-02-19: at java.util.TimerThread.mainLoop(Timer.java:512) log.2009-12-02-19: at java.util.TimerThread.run(Timer.java:462) log.2009-12-02-19: log.2009-12-02-23: WARN [Timer-0] 2009-12-02 23:10:27,128 LoadDisseminator.java (line 44) Exception was generated at : 12/02/2009 23:10:27 on thread Timer-0 log.2009-12-02-23: log.2009-12-02-23:java.lang.NullPointerException log.2009-12-02-23: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:198) log.2009-12-02-23: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:200) log.2009-12-02-23: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpaceForPath(FileUtils.java:200) log.2009-12-02-23: at org.apache.cassandra.utils.FileUtils.getUsedDiskSpace(FileUtils.java:211) log.2009-12-02-23: at org.apache.cassandra.service.StorageService.getLoad(StorageService.java:510) log.2009-12-02-23: at org.apache.cassandra.service.LoadDisseminator.run(LoadDisseminator.java:37) log.2009-12-02-23: at java.util.TimerThread.mainLoop(Timer.java:512) log.2009-12-02-23: at java.util.TimerThread.run(Timer.java:462) log.2009-12-02-23: Recall that these line numbers should match the 29 Nov build. Chris Goffinet [[email protected]] said: >After you stop the test. Do you see the pending tasks for compaction >drop? I see chatter in the log files saying that compactions are happening, but the number of reported pending tasks has increased while I was watching. Chris Goffinet [[email protected]] said: >[Were] there any other numbers in tpstats from nodeprobe that are growing? I didn't ask this question before stopping my client. Now tpstats reports that there are no pending tasks other than 1672 COMPACTION-POOL tasks, and the only active tasks are one COMMITLOG and one COMPACTION-POOL. >Can you plot the number of SSTables? I suppose so. Do you mean the number of files, or one of the stats output from "nodeprobe tpstats"? Keep in mind that the number of files is influenced by the garbage collector, so I doubt that counting the files will tell us much. >Are you using the standard storage-conf.xml defaults? Pretty much. I attached the file to this email. My summary of the changes from 1 Dec 09 is still valid even though I changed Cassandra versions: >Briefly, it has default values, except RpcTimeoutInMillis is 30000 and the >partitioner is OrderPreservingPartitioner. Tim Freeman Email: [email protected] Desk in Palo Alto: (650) 857-2581 Home: (408) 774-1298 Cell: (408) 348-7536 (No reception business hours Monday, Tuesday, and Thursday; call my desk instead.) -----Original Message----- From: Jonathan Ellis [mailto:[email protected]] Sent: Thursday, December 03, 2009 11:14 AM To: [email protected] Subject: Re: Persistently increasing read latency On Thu, Dec 3, 2009 at 1:11 PM, Freeman, Tim <[email protected]> wrote: >>how many are in your data directories? is your compaction >>lagging 1000s of tables behind again? > > Yes, there are 2348 files in data/Keyspace1, and jconsole says the compaction > pool has >1600 pending tasks. If you stop doing client requests does that number start to go down? If not there is a bug. If there are exceptions in the logs then it is an obvious one otherwise something more subtle. :) -Jonathan
<!-- ~ Licensed to the Apache Software Foundation (ASF) under one ~ or more contributor license agreements. See the NOTICE file ~ distributed with this work for additional information ~ regarding copyright ownership. The ASF licenses this file ~ to you under the Apache License, Version 2.0 (the ~ "License"); you may not use this file except in compliance ~ with the License. You may obtain a copy of the License at ~ ~ http://www.apache.org/licenses/LICENSE-2.0 ~ ~ Unless required by applicable law or agreed to in writing, ~ software distributed under the License is distributed on an ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY ~ KIND, either express or implied. See the License for the ~ specific language governing permissions and limitations ~ under the License. --> <Storage> <!--======================================================================--> <!-- Basic Configuration --> <!--======================================================================--> <!-- ~ The name of this cluster. This is mainly used to prevent machines in ~ one logical cluster from joining another. --> <ClusterName>Test Cluster</ClusterName> <!-- ~ Turn on to make new [non-seed] nodes automatically migrate the right data ~ to themselves. (If no InitialToken is specified, they will pick one ~ such that they will get half the range of the most-loaded node.) ~ If a node starts up without bootstrapping, it will mark itself bootstrapped ~ so that you can't subsequently accidently bootstrap a node with ~ data on it. (You can reset this by wiping your data and commitlog ~ directories.) ~ ~ Off by default so that new clusters and upgraders from 0.4 don't ~ bootstrap immediately. You should turn this on when you start adding ~ new nodes to a cluster that already has data on it. (If you are upgrading ~ from 0.4, start your cluster with it off once before changing it to true. ~ Otherwise, no data will be lost but you will incur a lot of unnecessary ~ I/O before your cluster starts up.) --> <AutoBootstrap>false</AutoBootstrap> <!-- ~ Keyspaces and ColumnFamilies: ~ A ColumnFamily is the Cassandra concept closest to a relational ~ table. Keyspaces are separate groups of ColumnFamilies. Except in ~ very unusual circumstances you will have one Keyspace per application. ~ There is an implicit keyspace named 'system' for Cassandra internals. --> <Keyspaces> <Keyspace Name="Keyspace1"> <!-- ~ The fraction of keys per sstable whose locations we keep in ~ memory in "mostly LRU" order. (JUST the key locations, NOT any ~ column values.) ~ The amount of memory used by the default setting of 0.01 is ~ comparable to the amount used by the internal per-sstable key ~ index. Consider increasing this if you have fewer, wider rows. ~ Set to 0 to disable entirely. --> <KeysCachedFraction>0.01</KeysCachedFraction> <!-- ~ The CompareWith attribute tells Cassandra how to sort the columns ~ for slicing operations. The default is BytesType, which is a ~ straightforward lexical comparison of the bytes in each column. ~ Other options are AsciiType, UTF8Type, LexicalUUIDType, TimeUUIDType, ~ and LongType. You can also specify the fully-qualified class ~ name to a class of your choice extending ~ org.apache.cassandra.db.marshal.AbstractType. ~ ~ SuperColumns have a similar CompareSubcolumnsWith attribute. ~ ~ BytesType: Simple sort by byte value. No validation is performed. ~ AsciiType: Like BytesType, but validates that the input can be ~ parsed as US-ASCII. ~ UTF8Type: A string encoded as UTF8 ~ LongType: A 64bit long ~ LexicalUUIDType: A 128bit UUID, compared lexically (by byte value) ~ TimeUUIDType: a 128bit version 1 UUID, compared by timestamp ~ ~ (To get the closest approximation to 0.3-style supercolumns, you ~ would use CompareWith=UTF8Type CompareSubcolumnsWith=LongType.) ~ ~ An optional `Comment` attribute may be used to attach additional ~ human-readable information about the column family to its definition. --> <ColumnFamily CompareWith="BytesType" Name="Standard1"/> <ColumnFamily CompareWith="UTF8Type" Name="Standard2"/> <ColumnFamily CompareWith="TimeUUIDType" Name="StandardByUUID1"/> <ColumnFamily ColumnType="Super" CompareWith="UTF8Type" CompareSubcolumnsWith="UTF8Type" Name="Super1" Comment="A column family with supercolumns, whose column and subcolumn names are UTF8 strings"/> </Keyspace> </Keyspaces> <!-- ~ Partitioner: any IPartitioner may be used, including your own as long ~ as it is on the classpath. Out of the box, Cassandra provides ~ org.apache.cassandra.dht.RandomPartitioner, ~ org.apache.cassandra.dht.OrderPreservingPartitioner, and ~ org.apache.cassandra.dht.CollatingOrderPreservingPartitioner. ~ (CollatingOPP colates according to EN,US rules, not naive byte ~ ordering. Use this as an example if you need locale-aware collation.) ~ Range queries require using an order-preserving partitioner. ~ ~ Achtung! Changing this parameter requires wiping your data ~ directories, since the partitioner can modify the sstable on-disk ~ format. --> <!-- If we want to be able to clear the table for tests, we need to be able to query the list of keys so we can delete them. This apparently requires a range query to get the list of keys, which requires an order-preserving partitioner. <Partitioner>org.apache.cassandra.dht.RandomPartitioner</Partitioner> --> <Partitioner>org.apache.cassandra.dht.OrderPreservingPartitioner</Partitioner> <!-- ~ If you are using an order-preserving partitioner and you know your key ~ distribution, you can specify the token for this node to use. (Keys ~ are sent to the node with the "closest" token, so distributing your ~ tokens equally along the key distribution space will spread keys ~ evenly across your cluster.) This setting is only checked the first ~ time a node is started. ~ This can also be useful with RandomPartitioner to force equal spacing ~ of tokens around the hash space, especially for clusters with a small ~ number of nodes. --> <InitialToken>0</InitialToken> <!-- ~ EndPointSnitch: Setting this to the class that implements ~ IEndPointSnitch which will see if two endpoints are in the same data ~ center or on the same rack. Out of the box, Cassandra provides ~ org.apache.cassandra.locator.EndPointSnitch --> <EndPointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndPointSnitch> <!-- ~ Strategy: Setting this to the class that implements ~ IReplicaPlacementStrategy will change the way the node picker works. ~ Out of the box, Cassandra provides ~ org.apache.cassandra.locator.RackUnawareStrategy and ~ org.apache.cassandra.locator.RackAwareStrategy (place one replica in ~ a different datacenter, and the others on different racks in the same ~ one.) --> <ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy> <!-- Number of replicas of the data --> <ReplicationFactor>1</ReplicationFactor> <!-- ~ Directories: Specify where Cassandra should store different data on ~ disk. Keep the data disks and the CommitLog disks separate for best ~ performance --> <CommitLogDirectory>/root/data/cassandra/commitlog</CommitLogDirectory> <DataFileDirectories> <DataFileDirectory>/root/data/cassandra/data</DataFileDirectory> </DataFileDirectories> <CalloutLocation>/root/data/cassandra/callouts</CalloutLocation> <StagingFileDirectory>/root/data/cassandra/staging</StagingFileDirectory> <!-- ~ Addresses of hosts that are deemed contact points. Cassandra nodes ~ use this list of hosts to find each other and learn the topology of ~ the ring. You must change this if you are running multiple nodes! ~ According to http://issues.apache.org/jira/browse/CASSANDRA-150 ~ and chat from jbellis on 23 Nov 09, you just want one or two ~ seeds, not all the nodes as seeds. But they checked in a fix ~ for the bug on 26 Nov 09, and this build is dated 29 Nov 09. ~ Might as well give their fix a whirl. --> <Seeds> <Seed>127.0.0.1</Seed> </Seeds> <!-- Miscellaneous --> <!-- Time to wait for a reply from other nodes before failing the command --> <!-- <RpcTimeoutInMillis>5000</RpcTimeoutInMillis> --> <!-- If I get too many timeouts, try increasing this. Tim Freeman 02 Dec 2009. I do get timeouts, with the typical timeout caused by an operation that takes 6711 ms when timeout happens at 5000 ms. --> <RpcTimeoutInMillis>30000</RpcTimeoutInMillis> <!-- Size to allow commitlog to grow to before creating a new segment --> <CommitLogRotationThresholdInMB>128</CommitLogRotationThresholdInMB> <!-- Local hosts and ports --> <!-- ~ Address to bind to and tell other nodes to connect to. You _must_ ~ change this if you want multiple nodes to be able to communicate! ~ ~ Leaving it blank leaves it up to InetAddress.getLocalHost(). This ~ will always do the Right Thing *if* the node is properly configured ~ (hostname, name resolution, etc), and the Right Thing is to use the ~ address associated with the hostname (it might not be). --> <ListenAddress>127.0.0.1</ListenAddress> <!-- TCP port, for commands and data --> <StoragePort>7000</StoragePort> <!-- UDP port, for membership communications (gossip) --> <ControlPort>7001</ControlPort> <!-- ~ The address to bind the Thrift RPC service to. Unlike ListenAddress ~ above, you *can* specify 0.0.0.0 here if you want Thrift to listen on ~ all interfaces. ~ ~ Leaving this blank has the same effect it does for ListenAddress, ~ (i.e. it will be based on the configured hostname of the node). --> <ThriftAddress>localhost</ThriftAddress> <!-- Thrift RPC port (the port clients connect to). --> <ThriftPort>9160</ThriftPort> <!-- ~ Whether or not to use a framed transport for Thrift. If this option ~ is set to true then you must also use a framed transport on the ~ client-side, (framed and non-framed transports are not compatible). --> <ThriftFramedTransport>false</ThriftFramedTransport> <!--======================================================================--> <!-- Memory, Disk, and Performance --> <!--======================================================================--> <!-- ~ Buffer size to use when performing contiguous column slices. Increase ~ this to the size of the column slices you typically perform. ~ (Name-based queries are performed with a buffer size of ~ ColumnIndexSizeInKB.) --> <SlicedBufferSizeInKB>64</SlicedBufferSizeInKB> <!-- ~ Buffer size to use when flushing memtables to disk. (Only one ~ memtable is ever flushed at a time.) Increase (decrease) the index ~ buffer size relative to the data buffer if you have few (many) ~ columns per key. Bigger is only better _if_ your memtables get large ~ enough to use the space. (Check in your data directory after your ~ app has been running long enough.) --> <FlushDataBufferSizeInMB>32</FlushDataBufferSizeInMB> <FlushIndexBufferSizeInMB>8</FlushIndexBufferSizeInMB> <!-- ~ Add column indexes to a row after its contents reach this size. ~ Increase if your column values are large, or if you have a very large ~ number of columns. The competing causes are, Cassandra has to ~ deserialize this much of the row to read a single column, so you want ~ it to be small - at least if you do many partial-row reads - but all ~ the index data is read for each access, so you don't want to generate ~ that wastefully either. --> <ColumnIndexSizeInKB>64</ColumnIndexSizeInKB> <!-- ~ The maximum amount of data to store in memory per ColumnFamily before ~ flushing to disk. Note: There is one memtable per column family, and ~ this threshold is based solely on the amount of data stored, not ~ actual heap memory usage (there is some overhead in indexing the ~ columns). --> <MemtableSizeInMB>64</MemtableSizeInMB> <!-- ~ The maximum number of columns in millions to store in memory per ~ ColumnFamily before flushing to disk. This is also a per-memtable ~ setting. Use with MemtableSizeInMB to tune memory usage. --> <MemtableObjectCountInMillions>0.1</MemtableObjectCountInMillions> <!-- ~ The maximum time to leave a dirty memtable unflushed. ~ (While any affected columnfamilies have unflushed data from a ~ commit log segment, that segment cannot be deleted.) ~ This needs to be large enough that it won't cause a flush storm ~ of all your memtables flushing at once because none has hit ~ the size or count thresholds yet. For production, a larger ~ value such as 1440 is recommended. --> <MemtableFlushAfterMinutes>60</MemtableFlushAfterMinutes> <!-- ~ Unlike most systems, in Cassandra writes are faster than reads, so ~ you can afford more of those in parallel. A good rule of thumb is 2 ~ concurrent reads per processor core. Increase ConcurrentWrites to ~ the number of clients writing at once if you enable CommitLogSync + ~ CommitLogSyncDelay. --> <ConcurrentReads>8</ConcurrentReads> <ConcurrentWrites>32</ConcurrentWrites> <!-- ~ CommitLogSync may be either "periodic" or "batch." When in batch ~ mode, Cassandra won't ack writes until the commit log has been ~ fsynced to disk. It will wait up to CommitLogSyncBatchWindowInMS ~ milliseconds for other writes, before performing the sync. ~ This is less necessary in Cassandra than in traditional databases ~ since replication reduces the odds of losing data from a failure ~ after writing the log entry but before it actually reaches the disk. ~ So the other option is "timed," where writes may be acked immediately ~ and the CommitLog is simply synced every CommitLogSyncPeriodInMS ~ milliseconds. --> <CommitLogSync>periodic</CommitLogSync> <!-- ~ Interval at which to perform syncs of the CommitLog in periodic mode. ~ Usually the default of 1000ms is fine; increase it only if the ~ CommitLog PendingTasks backlog in jmx shows that you are frequently ~ scheduling a second sync while the first has not yet been processed. --> <CommitLogSyncPeriodInMS>1000</CommitLogSyncPeriodInMS> <!-- ~ Delay (in milliseconds) during which additional commit log entries ~ may be written before fsync in batch mode. This will increase ~ latency slightly, but can vastly improve throughput where there are ~ many writers. Set to zero to disable (each entry will be synced ~ individually). Reasonable values range from a minimal 0.1 to 10 or ~ even more if throughput matters more than latency. --> <!-- <CommitLogSyncBatchWindowInMS>1</CommitLogSyncBatchWindowInMS> --> <!-- ~ Time to wait before garbage-collection deletion markers. Set this to ~ a large enough value that you are confident that the deletion marker ~ will be propagated to all replicas by the time this many seconds has ~ elapsed, even in the face of hardware failures. The default value is ~ ten days. --> <GCGraceSeconds>864000</GCGraceSeconds> <!-- ~ The threshold size in megabytes the binary memtable must grow to, ~ before it's submitted for flushing to disk. --> <BinaryMemtableSizeInMB>256</BinaryMemtableSizeInMB> </Storage>
