cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Gary Dusbabek (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-1715) More schema migration race conditions
Date Tue, 09 Nov 2010 14:04:12 GMT

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

Gary Dusbabek commented on CASSANDRA-1715:
------------------------------------------

bq. UpdateColumnFamily doesn't acquireLocks().
It does on line 82 in my checkout.
bq. Shouldn't Migration do that so the subclasses don't have to?
It doesn't make sense to lock on the Add* methods, but I agree it might be easier just to
do the locking in the superclass. What do you think?

> More schema migration race conditions
> -------------------------------------
>
>                 Key: CASSANDRA-1715
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1715
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.7 beta 1
>            Reporter: Jonathan Ellis
>            Assignee: Gary Dusbabek
>            Priority: Critical
>             Fix For: 0.7.0
>
>         Attachments: v1-0001-take-drop-off-CompactionManager.txt, v1-0002-compaction-lock.txt,
v1-0003-migration-uses-locks.txt, v1-0004-handle-moved-dropped-CF-prior-to-pending-compaction-st.txt
>
>
> Related to CASSANDRA-1631.
> This is still a bug with schema updates to an existing CF, since reloadCf is doing a
unload/init cycle. So flushing + compaction is an issue there as well. Here is a stacktrace
from during an index creation where it stubbed its toe on an incomplete sstable from an in-progress
compaction (path names anonymized):
> {code}
> INFO [CompactionExecutor:1] 2010-11-02 16:31:00,553 CompactionManager.java (line 224)
Compacting [org.apache.cassandra.io.sstable.SSTableReader(path='Standard1-e-6-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='Standard1-e-7-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='Standard1-e-8-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='Standard1-e-9-Data.db')]
> ...
> ERROR [MigrationStage:1] 2010-11-02 16:31:10,939 ColumnFamilyStore.java (line 244) Corrupt
sstable Standard1-tmp-e-10-<>=[Data.db, Index.db]; skipped
> java.io.EOFException
>         at org.apache.cassandra.utils.FBUtilities.skipShortByteArray(FBUtilities.java:308)
>         at org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:231)
>         at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:286)
>         at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:202)
>         at org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:235)
>         at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:443)
>         at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:431)
>         at org.apache.cassandra.db.Table.initCf(Table.java:335)
>         at org.apache.cassandra.db.Table.reloadCf(Table.java:343)
>         at org.apache.cassandra.db.migration.UpdateColumnFamily.applyModels(UpdateColumnFamily.java:89)
>         at org.apache.cassandra.db.migration.Migration.apply(Migration.java:158)
>         at org.apache.cassandra.thrift.CassandraServer$2.call(CassandraServer.java:672)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:619)
> ...
>  INFO [CompactionExecutor:1] 2010-11-02 16:31:31,970 CompactionManager.java (line 303)
Compacted to Standard1-tmp-e-10-Data.db.  213,657,983 to 213,657,983 (~100% of original) bytes
for 626,563 keys.  Time: 31,416ms.
> {code}
> There is also a race between schema modification and streaming.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message