[ 
https://issues.apache.org/jira/browse/CASSANDRA-8631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14280444#comment-14280444
 ] 

Benedict commented on CASSANDRA-8631:
-------------------------------------

Do we need this test, or want to fix it? The problem is almost certainly that 
1s is not frequent enough to keep anywhere near our goal memory limit when 
running on a local only spam test. Since we only enforce memory constraints in 
2.0 by the flush queue length (by default, iirc, this is 4, so 4s have to 
elapse before we actually start _blocking_ writes) this happily allows us to go 
well over limit. It's generally a known problem in 2.0. To make it work we need 
to artificially run MeteredFlusher (MUCH) more frequently, at which point we 
aren't testing C* as it actually runs. Alternatively, we could fix a rate of 
work production that should guarantee 1s is often enough to keep us within our 
limits, but this doesn't really perform much of use that a simple unit test 
couldn't - it just checks metered flusher runs, not that it enforces anything 
much.

Personally I think we should just drop this test from 2.0, since we know 2.0's 
metered flusher approach is a bit broken. The test functions as it was 
originally designed in 2.1, because memory limits are directly enforced, 
whereas designing a _useful_ test that also works with 2.0's semantics seems 
nontrivial. However making either of the two suggested modifications so it at 
least _appears_ to do something useful _is_ pretty trivial. So if somebody has 
a strong opinion on this (I don't really), please chime in!

> long-test MeteredFlusherTest fails with heap OOM
> ------------------------------------------------
>
>                 Key: CASSANDRA-8631
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8631
>             Project: Cassandra
>          Issue Type: Test
>          Components: Tests
>         Environment: cassandra-2.0 commit 90780b5
>            Reporter: Michael Shuler
>            Assignee: Benedict
>            Priority: Minor
>              Labels: unit-test
>
> I attempted to find a spot where this test actually passes, and I was unable 
> to do so. I tried back to 1.1 HEAD and various release tags under jdk7 and 
> jdk6 for <= 1.2 tags. In 1.2.0 with jdk6, the test times out with 
> java.lang.OutOfMemoryError: GC overhead limit exceeded, so it's similar.
> {noformat}
>     [junit] Testcase: 
> testManyMemtables(org.apache.cassandra.db.MeteredFlusherTest):    Caused an 
> ERROR
>     [junit] Java heap space
>     [junit] java.lang.OutOfMemoryError: Java heap space
>     [junit]     at java.nio.HeapByteBuffer.<init>(HeapByteBuffer.java:57)
>     [junit]     at java.nio.ByteBuffer.allocate(ByteBuffer.java:331)
>     [junit]     at 
> org.apache.cassandra.utils.SlabAllocator$Region.<init>(SlabAllocator.java:157)
>     [junit]     at 
> org.apache.cassandra.utils.SlabAllocator$Region.<init>(SlabAllocator.java:131)
>     [junit]     at 
> org.apache.cassandra.utils.SlabAllocator.getRegion(SlabAllocator.java:101)
>     [junit]     at 
> org.apache.cassandra.utils.SlabAllocator.allocate(SlabAllocator.java:73)
>     [junit]     at 
> org.apache.cassandra.utils.Allocator.clone(Allocator.java:30)
>     [junit]     at org.apache.cassandra.db.Column.localCopy(Column.java:277)
>     [junit]     at org.apache.cassandra.db.Memtable$1.apply(Memtable.java:114)
>     [junit]     at org.apache.cassandra.db.Memtable$1.apply(Memtable.java:111)
>     [junit]     at 
> org.apache.cassandra.db.AtomicSortedColumns.addAllWithSizeDelta(AtomicSortedColumns.java:194)
>     [junit]     at org.apache.cassandra.db.Memtable.resolve(Memtable.java:218)
>     [junit]     at org.apache.cassandra.db.Memtable.put(Memtable.java:165)
>     [junit]     at 
> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:901)
>     [junit]     at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:374)
>     [junit]     at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:339)
>     [junit]     at 
> org.apache.cassandra.db.RowMutation.applyUnsafe(RowMutation.java:216)
>     [junit]     at 
> org.apache.cassandra.db.MeteredFlusherTest.testManyMemtables(MeteredFlusherTest.java:59)
>     [junit] 
>     [junit] 
>     [junit] Test org.apache.cassandra.db.MeteredFlusherTest FAILED
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to