[
https://issues.apache.org/jira/browse/CASSANDRA-10421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14968459#comment-14968459
]
Stefania commented on CASSANDRA-10421:
--------------------------------------
[~aweisberg] : one more
[commit|https://github.com/stef1927/cassandra/commit/bf18a71be49d4b6cd9242e97846fcf69653a53cd]
to review.
I found some exceptions reported for the compressed versions of the tests that
were timing out. Unsure why this behavior on Jenkins, that is why the
compressed tests were not reported as failing whereas the normal tests were
reported as timing out. However, the same identical thing happened twice with
{{MoveTest}} and {{CellTest}}, so I am pretty sure the cause of the timeouts
were the exceptions below, which I've fixed. I've started a new CI run.
Here are the exceptions:
{code}
WARN [MemtableFlushWriter:1] 2015-10-19 01:29:08,076
open(build/test/cassandra/data:241/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6,
O_RDONLY) failed, errno (2).
ERROR [MemtableFlushWriter:1] 2015-10-19 01:29:08,102 Fatal exception in thread
Thread[MemtableFlushWriter:1,5,main]
java.lang.NullPointerException: null
at java.util.Objects.requireNonNull(Objects.java:203) ~[na:1.8.0_45]
at java.util.Arrays$ArrayList.<init>(Arrays.java:3813) ~[na:1.8.0_45]
at java.util.Arrays.asList(Arrays.java:3800) ~[na:1.8.0_45]
at
org.apache.cassandra.db.lifecycle.LogRecord.getExistingFiles(LogRecord.java:243)
~[main/:na]
at org.apache.cassandra.db.lifecycle.LogRecord.make(LogRecord.java:129)
~[main/:na]
at
org.apache.cassandra.db.lifecycle.LogFile.makeRecord(LogFile.java:272)
~[main/:na]
at org.apache.cassandra.db.lifecycle.LogFile.add(LogFile.java:262)
~[main/:na]
at
org.apache.cassandra.db.lifecycle.LogTransaction.trackNew(LogTransaction.java:133)
~[main/:na]
at
org.apache.cassandra.db.lifecycle.LifecycleTransaction.trackNew(LifecycleTransaction.java:517)
~[main/:na]
at
org.apache.cassandra.io.sstable.format.big.BigTableWriter.<init>(BigTableWriter.java:68)
~[main/:na]
at
org.apache.cassandra.io.sstable.format.big.BigFormat$WriterFactory.open(BigFormat.java:93)
~[main/:na]
at
org.apache.cassandra.io.sstable.format.SSTableWriter.create(SSTableWriter.java:96)
~[main/:na]
at
org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter.create(SimpleSSTableMultiWriter.java:114)
~[main/:na]
at
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.createSSTableMultiWriter(AbstractCompactionStrategy.java:514)
~[main/:na]
at
org.apache.cassandra.db.compaction.CompactionStrategyManager.createSSTableMultiWriter(CompactionStrategyManager.java:506)
~[main/:na]
at
org.apache.cassandra.db.ColumnFamilyStore.createSSTableMultiWriter(ColumnFamilyStore.java:472)
~[main/:na]
at
org.apache.cassandra.db.Memtable$FlushRunnable.createFlushWriter(Memtable.java:429)
~[main/:na]
at
org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:366)
~[main/:na]
at
org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:352)
~[main/:na]
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
~[main/:na]
at
com.google.common.util.concurrent.MoreExecutors$DirectExecutorService.execute(MoreExecutors.java:299)
~[guava-18.0.jar:na]
at
org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1037)
~[main/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
~[na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_45]
ERROR [Reference-Reaper:1] 2015-10-19 01:29:11,444 LEAK DETECTED: a reference
(org.apache.cassandra.utils.concurrent.Ref$State@60b352b3) to class
org.apache.cassandra.db.lifecycle.LogTransaction$TransactionTidier@632410305:[build/test/cassandra/data:241/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/ma_txn_flush_cadfa480-7600-11e5-a7c7-1d8546d0512d.log]
was not released before the reference was garbage collected
ERROR [Reference-Reaper:1] 2015-10-19 01:29:11,445 Allocate trace
org.apache.cassandra.utils.concurrent.Ref$State@60b352b3:
Thread[MemtableFlushWriter:1,5,main]
at java.lang.Thread.getStackTrace(Thread.java:1552)
at org.apache.cassandra.utils.concurrent.Ref$Debug.<init>(Ref.java:218)
at org.apache.cassandra.utils.concurrent.Ref$State.<init>(Ref.java:148)
at org.apache.cassandra.utils.concurrent.Ref.<init>(Ref.java:70)
at
org.apache.cassandra.db.lifecycle.LogTransaction.<init>(LogTransaction.java:122)
at
org.apache.cassandra.db.lifecycle.LifecycleTransaction.offline(LifecycleTransaction.java:147)
at
org.apache.cassandra.db.Memtable$FlushRunnable.createFlushWriter(Memtable.java:426)
at
org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:366)
at
org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:352)
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at
com.google.common.util.concurrent.MoreExecutors$DirectExecutorService.execute(MoreExecutors.java:299)
at
org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1037)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
ERROR [Reference-Reaper:1] 2015-10-19 01:29:11,445 Error when closing class
org.apache.cassandra.db.lifecycle.LogTransaction$TransactionTidier@632410305:[build/test/cassandra/data:241/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/ma_txn_flush_cadfa480-7600-11e5-a7c7-1d8546d0512d.log]
java.lang.AssertionError: Expected a completed transaction:
[build/test/cassandra/data:241/system_schema/keyspaces-abac5682dea631c5b535b3d6cffd0fb6/ma_txn_flush_cadfa480-7600-11e5-a7c7-1d8546d0512d.log]
at
org.apache.cassandra.db.lifecycle.LogTransaction$TransactionTidier.run(LogTransaction.java:243)
~[main/:na]
at
org.apache.cassandra.db.lifecycle.LogTransaction$TransactionTidier.tidy(LogTransaction.java:230)
~[main/:na]
at
org.apache.cassandra.utils.concurrent.Ref$GlobalState.release(Ref.java:294)
~[main/:na]
at
org.apache.cassandra.utils.concurrent.Ref$State.release(Ref.java:193)
~[main/:na]
at
org.apache.cassandra.utils.concurrent.Ref$ReferenceReaper.run(Ref.java:342)
[main/:na]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_45]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_45]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
{code}
> Potential issue with LogTransaction as it only checks in a single directory
> for files
> -------------------------------------------------------------------------------------
>
> Key: CASSANDRA-10421
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10421
> Project: Cassandra
> Issue Type: Bug
> Reporter: Marcus Eriksson
> Assignee: Stefania
> Priority: Blocker
> Fix For: 3.0.0
>
>
> When creating a new LogTransaction we try to create the new logfile in the
> same directory as the one we are writing to, but as we use
> {{[directories.getDirectoryForNewSSTables()|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java#L125]}}
> this might end up in "any" of the configured data directories. If it does,
> we will not be able to clean up leftovers as we check for files in the same
> directory as the logfile was created:
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/lifecycle/LogRecord.java#L163
> cc [~Stefania]
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)