If its determined that this is due to a very large row, what are my options?

Thanks

On 3/5/11 7:11 PM, aaron morton wrote:
First question is which version are you running ? Am guessing 0.6 something

If you have OOM in the compaction thread it may be because of a very large row. The CF information available through JConsole will give you the max row size for the CF.

Your setting for RowWarningThresholdInMB is 512, however the default setting in 0.6.12 is 64mb. Here's the inline help from storage-conf.xml for 0.6.12

<!--
   ~ Size of compacted row above which to log a warning.  If compacted
   ~ rows do not fit in memory, Cassandra will crash.  (This is explained
   ~ in http://wiki.apache.org/cassandra/CassandraLimitations and is
   ~ scheduled to be fixed in 0.7.)  Large rows can also be a problem
   ~ when row caching is enabled.
  -->
<RowWarningThresholdInMB>64</RowWarningThresholdInMB>

In 0.7 the equivalent setting is in_memory_compaction_limit_in_mb , here's the file help from the 0.7.3 conf file

# Size limit for rows being compacted in memory.  Larger rows will spill
# over to disk and use a slower two-pass compaction process.  A message
# will be logged specifying the row key.
in_memory_compaction_limit_in_mb: 64
Hope that helps.
Aaron


On 5/03/2011, at 10:54 AM, Mark wrote:

Thats very nice of you. Thanks

<Storage>
<ClusterName>MyCluster</ClusterName>
<AutoBootstrap>true</AutoBootstrap>
<HintedHandoffEnabled>true</HintedHandoffEnabled>
<IndexInterval>128</IndexInterval>

<Keyspaces>
<Keyspace Name="MyCompany">
<ColumnFamily Name="SearchLog"
                    ColumnType="Super"
                    CompareWith="TimeUUIDType"
                    CompareSubcolumnsWith="BytesType"/>
<ColumnFamily Name="ProductLog"
                    ColumnType="Super"
                    CompareWith="TimeUUIDType"
                    CompareSubcolumnsWith="BytesType"/>
<ColumnFamily Name="RequestLog"
                    ColumnType="Super"
                    CompareWith="TimeUUIDType"
                    CompareSubcolumnsWith="BytesType"/>
<ColumnFamily Name="ClickLog"
                    ColumnType="Super"
                    CompareWith="TimeUUIDType"
                    CompareSubcolumnsWith="BytesType"/>
<ColumnFamily Name="ItemTranslation"
                    ColumnType="Super"
                    CompareWith="BytesType"
                    CompareSubcolumnsWith="BytesType"/>
<ColumnFamily Name="ItemTranslationIndex"
                    CompareWith="BytesType"/>
<ColumnFamily Name="RelatedObject"
                    CompareWith="LongType"/>

<ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
<ReplicationFactor>2</ReplicationFactor>
<EndPointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndPointSnitch>
</Keyspace>
</Keyspaces>

<Authenticator>org.apache.cassandra.auth.AllowAllAuthenticator</Authenticator>
<Partitioner>org.apache.cassandra.dht.RandomPartitioner</Partitioner>
<InitialToken></InitialToken>

<SavedCachesDirectory>/var/lib/cassandra/saved_caches</SavedCachesDirectory>
<CommitLogDirectory>/var/lib/cassandra/commitlog</CommitLogDirectory>
<DataFileDirectories>
<DataFileDirectory>/var/lib/cassandra/data</DataFileDirectory>
</DataFileDirectories>

<Seeds>
<Seed>cassandra1</Seed>
<Seed>cassandra2</Seed>
</Seeds>

<RpcTimeoutInMillis>10000</RpcTimeoutInMillis>
<CommitLogRotationThresholdInMB>128</CommitLogRotationThresholdInMB>

<ListenAddress></ListenAddress>
<StoragePort>7000</StoragePort>

<ThriftAddress></ThriftAddress>
<ThriftPort>9160</ThriftPort>
<ThriftFramedTransport>false</ThriftFramedTransport>

<DiskAccessMode>mmap_index_only</DiskAccessMode>

<RowWarningThresholdInMB>512</RowWarningThresholdInMB>
<SlicedBufferSizeInKB>64</SlicedBufferSizeInKB>
<FlushDataBufferSizeInMB>32</FlushDataBufferSizeInMB>
<FlushIndexBufferSizeInMB>8</FlushIndexBufferSizeInMB>
<ColumnIndexSizeInKB>64</ColumnIndexSizeInKB>
<MemtableThroughputInMB>64</MemtableThroughputInMB>
<BinaryMemtableThroughputInMB>256</BinaryMemtableThroughputInMB>
<MemtableFlushAfterMinutes>60</MemtableFlushAfterMinutes>

<ConcurrentReads>8</ConcurrentReads>
<ConcurrentWrites>32</ConcurrentWrites>

<CommitLogSync>periodic</CommitLogSync>
<CommitLogSyncPeriodInMS>10000</CommitLogSyncPeriodInMS>
<GCGraceSeconds>864000</GCGraceSeconds>

<DoConsistencyChecksBoolean>true</DoConsistencyChecksBoolean>
</Storage>


On 3/4/11 1:05 PM, Narendra Sharma wrote:
I have been through tuning for GC and OOM recently. If you can provide the cassandra.yaml, I can help. Mostly I had to play with memtable thresholds.

Thanks,
Naren

On Fri, Mar 4, 2011 at 12:43 PM, Mark <static.void....@gmail.com <mailto:static.void....@gmail.com>> wrote:

    We have 7 column families and we are not using the default key
    cache (200000).

    These were our initial settings so it was not in response to
    anything. Would you recommend anything else? Thanks



    On 3/4/11 12:34 PM, Chris Burroughs wrote:

        - Are you using a key cache?  How many keys do you have?
         Across how
        many column families

        You configuration is unusual both in terms of not setting
        min heap ==
        max heap and the percentage of available RAM used for the
        heap.  Did you
        change the heap size in response to errors or for another
        reason?

        On 03/04/2011 03:25 PM, Mark wrote:

            This happens during compaction and we are not using the
            RowsCached
            attribute.

            Our initial/max heap are 2 and 6 respectively and we
            have 8 gigs in
            these machines.

            Thanks

            On 3/4/11 12:05 PM, Chris Burroughs wrote:

                - Does this occur only during compaction or at
                seemingly random times?
                - How large is your heap?  What jvm settings are you
                using? How much
                physical RAM do you have?
                - Do you have the row and/or key cache enabled?  How
                are they
                configured?  How large are they when the OOM is thrown?

                On 03/04/2011 02:38 PM, Mark Miller wrote:

                    Other than adding more memory to the machine is
                    there a way to solve
                    this? Please help. Thanks

                    ERROR [COMPACTION-POOL:1] 2011-03-04
                    11:11:44,891 CassandraDaemon.java
                    (line
                    org.apache.cassandra.thrift.CassandraDaemon$1)
                    Uncaught exception
                    in thread Thread[COMPACTION-POOL:1,5,main]
                    java.lang.OutOfMemoryError: Java heap space
                         at java.util.Arrays.copyOf(Arrays.java:2798)
                         at
                    
java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:111)
                         at
                    java.io.DataOutputStream.write(DataOutputStream.java:107)
                         at
                    java.io.FilterOutputStream.write(FilterOutputStream.java:97)
                         at
                    
org.apache.cassandra.utils.FBUtilities.writeByteArray(FBUtilities.java:298)

                         at
                    
org.apache.cassandra.db.ColumnSerializer.serialize(ColumnSerializer.java:66)


                         at
                    
org.apache.cassandra.db.SuperColumnSerializer.serialize(SuperColumn.java:311)


                         at
                    
org.apache.cassandra.db.SuperColumnSerializer.serialize(SuperColumn.java:284)


                         at
                    
org.apache.cassandra.db.ColumnFamilySerializer.serializeForSSTable(ColumnFamilySerializer.java:87)


                         at
                    
org.apache.cassandra.db.ColumnFamilySerializer.serializeWithIndexes(ColumnFamilySerializer.java:99)


                         at
                    
org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:140)


                         at
                    
org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.java:43)


                         at
                    
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:73)


                         at
                    
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)


                         at
                    
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)


                         at
                    
org.apache.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183)


                         at
                    
org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94)


                         at
                    
org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:294)


                         at
                    
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:101)


                         at
                    
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:82)

                         at
                    
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
                         at
                    java.util.concurrent.FutureTask.run(FutureTask.java:166)
                         at
                    
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)


                         at
                    
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)


                         at java.lang.Thread.run(Thread.java:636)



Reply via email to