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

Yuki Morishita commented on CASSANDRA-12203:
--------------------------------------------

I have one easily reproducable case for this. I tested with the following step 
in cassandra-2.1 branch and updated to cassandra-3.0 branch, and got the same 
exception as OP.

While in cassandra-2.1:

1. Create schema as follows:

{code:sql}
CREATE KEYSPACE IF NOT EXISTS ks WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 1};
CREATE TABLE IF NOT EXISTS ks.test (
  k   ascii,
  c1  ascii,
  c2  int,
  c3  int,
  val text,
  PRIMARY KEY (k, c1, c2, c3)
);
{code}

2. Create SSTable that contains several RangeTombstones:

{noformat}
$ ccm node1 cqlsh
Connected to 12203 at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 2.1.17-SNAPSHOT | CQL spec 3.2.1 | Native protocol v3]
Use HELP for help.
cqlsh> DELETE FROM ks.test WHERE k = 'a' AND c1 = 'a';
cqlsh> DELETE FROM ks.test WHERE k = 'a' AND c1 = 'a' AND c2 = 1;
cqlsh>
{noformat}

and then flush:

{noformat}
$ ccm node1 flush ks test
{noformat}

You should have SSTable like this:

{noformat}
$ ccm node1 json -k ks -c test
running
['/home/yuki/.ccm/12203/node1/data0/ks/test-17e3f3f0c95f11e69ed29dc952593398/ks-test-ka-1-Data.db']
-- ks-test-ka-1-Data.db -----
[
{"key": "a",
 "cells": [["a:_","a:1:_",1482532105311908,"t",1482532105],
           ["a:1:_","a:1:!",1482532109564587,"t",1482532109],
           ["a:1:!","a:!",1482532105311908,"t",1482532105]]}
]
{noformat}

Now upgrade to cassandra-3.0 branch.

This AE can be triggered in simple SELECT query, or any compaction/streaming 
operation.

{noformat}
$ ccm node1 cqlsh
Connected to 12203 at 127.0.0.1:9042.
[cqlsh 5.0.1 | Cassandra 3.0.11-SNAPSHOT | CQL spec 3.4.0 | Native protocol v4]
Use HELP for help.
cqlsh> SELECT * FROM ks.test;
ReadFailure: Error from server: code=1300 [Replica(s) failed to execute read] 
message="Operation failed - received 0 responses and 1 failures" 
info={'failures': 1, 'received_responses': 0, 'required_responses': 1, 
'consistency': 'ONE'}
cqlsh>
{noformat}

In system.log you can see:

{noformat}
WARN  [SharedPool-Worker-2] 2016-12-23 16:29:26,377 
AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread 
Thread[SharedPool-Worker-2,5,main]: {}
java.lang.AssertionError: null
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$TombstoneTracker.openNew(UnfilteredDeserializer.java:664)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:427)
 ~[main/:na]
        at 
org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:290)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.computeNext(SSTableIterator.java:126)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:153)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:340)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.AbstractSSTableIterator.hasNext(AbstractSSTableIterator.java:219)
 ~[main/:na]
        at 
org.apache.cassandra.db.columniterator.SSTableIterator.hasNext(SSTableIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:95)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
 ~[main/:na]
        at 
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
~[main/:na]
        at 
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) 
~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
 ~[main/:na]
        at 
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77)
 ~[main/:na]
        at 
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:300)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:145)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:138)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:134)
 ~[main/:na]
        at 
org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:76) 
~[main/:na]
        at 
org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:321) 
~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1796)
 ~[main/:na]
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2472)
 ~[main/:na]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_112]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
 ~[main/:na]
        at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
 [main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
[main/:na]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_112]
{noformat}

(Line number above is different as I put some debug logging in the code)

Taking a look at where AE is thrown:

{code}
assert metadata.comparator.compare(tombstone.start.bound, first.stop.bound) > 0;
{code}

This assertion is not be able to be satisfied for the SSTable above. Maybe we 
should change {{>}} to {{>=}} ?

/cc [~slebresne]

> AssertionError on compaction after upgrade (2.1.9 -> 3.7)
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-12203
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12203
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Cassandra 3.7 (upgrade from 2.1.9)
> Java version "1.8.0_91"
> Ubuntu 14.04.4 LTS (GNU/Linux 3.13.0-83-generic x86_64)
>            Reporter: Roman S. Borschel
>             Fix For: 3.0.x, 3.x
>
>
> After upgrading a Cassandra cluster from 2.1.9 to 3.7, one column family 
> (using SizeTieredCompaction) repeatedly and continuously failed compaction 
> (and thus also repair) across the cluster, with all nodes producing the 
> following errors in the logs:
> {noformat}
> 016-07-14T09:29:47.96855 |srv=cassandra|ERROR: Exception in thread 
> Thread[CompactionExecutor:3,1,main]
> 2016-07-14T09:29:47.96858 |srv=cassandra|java.lang.AssertionError: null
> 2016-07-14T09:29:47.96859 |srv=cassandra|   at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$TombstoneTracker.openNew(UnfilteredDeserializer.java:650)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96860 |srv=cassandra|   at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer$UnfilteredIterator.hasNext(UnfilteredDeserializer.java:423)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96860 |srv=cassandra|   at 
> org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.hasNext(UnfilteredDeserializer.java:298)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96860 |srv=cassandra|   at 
> org.apache.cassandra.io.sstable.SSTableSimpleIterator$OldFormatIterator.readStaticRow(SSTableSimpleIterator.java:133)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96861 |srv=cassandra|   at 
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdentityIterator.java:57)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96861 |srv=cassandra|   at 
> org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator$1.initializeIterator(BigTableScanner.java:334)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96862 |srv=cassandra|   at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96862 |srv=cassandra|   at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.isReverseOrder(LazilyInitializedUnfilteredRowIterator.java:70)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96863 |srv=cassandra|   at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$1.reduce(UnfilteredPartitionIterators.java:109)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96863 |srv=cassandra|   at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$1.reduce(UnfilteredPartitionIterators.java:100)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96864 |srv=cassandra|   at 
> org.apache.cassandra.utils.MergeIterator$Candidate.consume(MergeIterator.java:408)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96864 |srv=cassandra|   at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:203)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96865 |srv=cassandra|   at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:156)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96865 |srv=cassandra|   at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96866 |srv=cassandra|   at 
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$2.hasNext(UnfilteredPartitionIterators.java:150)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96866 |srv=cassandra|   at 
> org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:72)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96867 |srv=cassandra|   at 
> org.apache.cassandra.db.compaction.CompactionIterator.hasNext(CompactionIterator.java:226)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96867 |srv=cassandra|   at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:182)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96867 |srv=cassandra|   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96868 |srv=cassandra|   at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:82)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96868 |srv=cassandra|   at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96869 |srv=cassandra|   at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:264)
>  ~[apache-cassandra-3.7.jar:3.7]
> 2016-07-14T09:29:47.96870 |srv=cassandra|   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_91]
> 2016-07-14T09:29:47.96870 |srv=cassandra|   at 
> java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_91]
> 2016-07-14T09:29:47.96870 |srv=cassandra|   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  ~[na:1.8.0_91]
> 2016-07-14T09:29:47.96871 |srv=cassandra|   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_91]
> 2016-07-14T09:29:47.96871 |srv=cassandra| at 
> java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> {noformat}
> Only 1 out of 14 tables was affected by this problem after the upgrade.
> The schema of the affected table looks like this:
> {noformat}
>         create columnfamily if not exists cassandra12203
>             ( field1   uuid
>             , field2  ascii
>             , field3    int
>             , field4   text
>             , primary key (field1, field2, field3)
>             );
> {noformat}



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

Reply via email to