cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sam Tunnicliffe (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6924) Data Inserted Immediately After Secondary Index Creation is not Indexed
Date Tue, 15 Apr 2014 10:50:17 GMT

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

Sam Tunnicliffe commented on CASSANDRA-6924:
--------------------------------------------

This doesn't seem like a regression as the repro script fails for me just as consistently
on 1.2.15 as it does on later versions.

The issue appears to be that when a ks or cf is dropped, we don't update system.IndexInfo
to remove the entry for the 2i. Then when the ks/cf & index are recreated, we treat the
index creation not as a brand new index, but as if we're restarting and linking in an existing
index to the cf. So we skip the buildIndexAsync call that we should make which is what causes
some entries to never get indexed. 

Fixing this so that we do clean up IndexInfo leads to us running into CASSANDRA-5202 on pre-2.1
branches. On 2.1, we see the issues mentioned in CASSANDRA-6959 so as Sylvain suggests there,
the test needs to be changed to wait for schema agreement. This can be acheived with a 1s
wait, or by actively testing for agreement. Now that the buildIndexAsync call is happening
on index initialisation, we can insert this wait in one of two places: between the index creation
and the inserts, or between the inserts and the reads. I've updated the dtest accordingly
and added another variant which drops just the cf, rather than the entire ks (https://github.com/riptano/cassandra-dtest/pull/40).
I do still see the errors from {{CommitLogSegmentManager}} on 2.1 detailed on CASSANDRA-6959
even after applying the patch attached to that issue.

Likewise, using Tyler's original repro script, a 1s sleep before commencing the reads is now
enough to ensure the run succeeds (on the 2.1 branch).

On trunk, I get completely different errors running both the dtest & repro.py, both with
and without the IndexInfo fix:
{code}
ERROR [Thrift:1] 2014-04-14 15:45:10,714 CustomTThreadPoolServer.java:212 - Error occurred
during processing of message.
java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.IllegalArgumentException:
fromIndex(34) > toIndex(25)
        at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:411) ~[main/:na]
        at org.apache.cassandra.service.MigrationManager.announce(MigrationManager.java:281)
~[main/:na]
        at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:242)
~[main/:na]
        at org.apache.cassandra.cql3.statements.CreateIndexStatement.announceMigration(CreateIndexStatement.java:141)
~[main/:na]
        at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:71)
~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:180)
~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:214) ~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:204) ~[main/:na]
        at org.apache.cassandra.thrift.CassandraServer.execute_cql3_query(CassandraServer.java:1973)
~[main/:na]
        at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4486)
~[thrift/:na]
        at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4470)
~[thrift/:na]
        at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.1.jar:0.9.1]
        at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.1.jar:0.9.1]
        at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:194)
~[main/:na]
        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]
Caused by: java.util.concurrent.ExecutionException: java.lang.IllegalArgumentException: fromIndex(34)
> toIndex(25)
        at java.util.concurrent.FutureTask.report(FutureTask.java:122) ~[na:1.7.0_51]
        at java.util.concurrent.FutureTask.get(FutureTask.java:188) ~[na:1.7.0_51]
        at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:407) ~[main/:na]
        ... 16 common frames omitted
Caused by: java.lang.IllegalArgumentException: fromIndex(34) > toIndex(25)
        at java.util.TimSort.rangeCheck(TimSort.java:921) ~[na:1.7.0_51]
        at java.util.TimSort.sort(TimSort.java:182) ~[na:1.7.0_51]
        at java.util.Arrays.sort(Arrays.java:727) ~[na:1.7.0_51]
        at org.apache.cassandra.db.ArrayBackedSortedColumns.sortCells(ArrayBackedSortedColumns.java:113)
~[main/:na]
        at org.apache.cassandra.db.ArrayBackedSortedColumns.maybeSortCells(ArrayBackedSortedColumns.java:103)
~[main/:na]
        at org.apache.cassandra.db.ArrayBackedSortedColumns.getColumnCount(ArrayBackedSortedColumns.java:313)
~[main/:na]
        at org.apache.cassandra.db.ColumnFamilySerializer.contentSerializedSize(ColumnFamilySerializer.java:117)
~[main/:na]
        at org.apache.cassandra.db.ColumnFamilySerializer.serializedSize(ColumnFamilySerializer.java:132)
~[main/:na]
        at org.apache.cassandra.db.Mutation$MutationSerializer.serializedSize(Mutation.java:337)
~[main/:na]
        at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:201) ~[main/:na]
        at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:193) ~[main/:na]
        at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:349) ~[main/:na]
        at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:328) ~[main/:na]
        at org.apache.cassandra.db.Mutation.apply(Mutation.java:205) ~[main/:na]
        at org.apache.cassandra.db.DefsTables.mergeSchema(DefsTables.java:170) ~[main/:na]
        at org.apache.cassandra.service.MigrationManager$2.runMayThrow(MigrationManager.java:299)
~[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]
        ... 3 common frames omitted

{code}

> Data Inserted Immediately After Secondary Index Creation is not Indexed
> -----------------------------------------------------------------------
>
>                 Key: CASSANDRA-6924
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6924
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Tyler Hobbs
>            Assignee: Sam Tunnicliffe
>             Fix For: 2.0.7
>
>         Attachments: 6924-2.1.txt, repro.py
>
>
> The head of the cassandra-1.2 branch (currently 1.2.16-tentative) contains a regression
from 1.2.15.  Data that is inserted immediately after secondary index creation may never get
indexed.
> You can reproduce the issue with a [pycassa integration test|https://github.com/pycassa/pycassa/blob/master/tests/test_autopacking.py#L793]
by running:
> {noformat}
> nosetests tests/test_autopacking.py:TestKeyValidators.test_get_indexed_slices
> {noformat}
> from the pycassa directory.
> The operation order goes like this:
> # create CF
> # create secondary index
> # insert data
> # query secondary index
> If a short sleep is added in between steps 2 and 3, the data gets indexed and the query
is successful.
> If a sleep is only added in between steps 3 and 4, some of the data is never indexed
and the query will return incomplete results.  This appears to be the case even if the sleep
is relatively long (30s), which makes me think the data may never get indexed.



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

Mime
View raw message