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

Sylvain Lebresne commented on CASSANDRA-6959:
---------------------------------------------

I haven't looked at the CommitLog assertion (I'll let somewhat more up to date 
on the last commit log changes look at it), but regarding the 2nd one, the one 
in ExtendedFilter, that's because the test tries to use a 2ndary index before 
it's creation has been propagated to all nodes. More precisely, the {{select 
count(*) from ks.cf WHERE col1='asdf'}} of the test reaches the coordinator 
node1, which has seen the index creation and so the query is considered valid, 
but when the underlying RangeCommand reaches the 2nd node, that one doesn't yet 
know about the index and so fallback to a sequential scan, but we don't 
currently support that kind of sequential scan with filters hence the exception 
(and truly, that was not what the user asked about anyway). If you add a small 
sleep after the index creation, that exception goes away in particular. The 
current status quo (CASSANDRA-5886) is that it's the client job to check for 
schema agreement if need be and not query a table/index that is not fully 
propagated, and in that sense, it is the test that should be fixed. Note that 
I'm not against trying to have a clearer error message but I'm not really sure 
there's a much better place for this, so I did added a somewhat more explicit 
message to the assertionError which hopefully is good enough. 

> Reusing Keyspace and CF names raises assertion errors
> -----------------------------------------------------
>
>                 Key: CASSANDRA-6959
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6959
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Ryan McGuire
>
> The [dtest I 
> introduced|https://github.com/riptano/cassandra-dtest/commit/36960090d219ab8dbc7f108faa91c3ea5cea2bec]
>  to test CASSANDRA-6924 introduces some log errors which I think may be 
> related to  CASSANDRA-5202. 
> On 2.1 :
> {code}
> ERROR [MigrationStage:1] 2014-03-31 14:36:43,463 
> CommitLogSegmentManager.java:306 - Failed waiting for a forced recycle of 
> in-use commit log segments
> java.lang.AssertionError: null
>         at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.forceRecycleAll(CommitLogSegmentManager.java:301)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:160)
>  [main/:na]
>         at 
> org.apache.cassandra.db.DefsTables.dropColumnFamily(DefsTables.java:497) 
> [main/:na]
>         at 
> org.apache.cassandra.db.DefsTables.mergeColumnFamilies(DefsTables.java:296) 
> [main/:na]
>         at 
> org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:181) [main/:na]
>         at 
> org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:49)
>  [main/:na]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> [main/:na]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> [na:1.7.0_51]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> [na:1.7.0_51]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  [na:1.7.0_51]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  [na:1.7.0_51]
>         at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
> {code}
> On 2.0: 
> {code}
> ERROR [ReadStage:3] 2014-03-31 13:28:11,014 CassandraDaemon.java (line 198) 
> Exception in thread Thread[ReadStage:3,5,main]
> java.lang.AssertionError
>         at 
> org.apache.cassandra.db.filter.ExtendedFilter$WithClauses.getExtraFilter(ExtendedFilter.java:258)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1744)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1699)
>         at 
> org.apache.cassandra.db.PagedRangeCommand.executeLocally(PagedRangeCommand.java:119)
>         at 
> org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:39)
>         at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
> {code}
> To reproduce, you many need to comment out the assertion in that test, as it 
> is not 100% reproducible on the first try.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to