cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Meg Mara <mm...@digitalriver.com>
Subject RE: 2.1 to 3.0 upgrade problems
Date Wed, 29 Mar 2017 15:56:50 GMT
Hi Eric,

I am not sure why the commit logs were growing so much but I do have some suggestions regarding
the Unexpected exception during request and about upgradesstables. I had recently upgraded
from 2.2.5 to 3.0.10 and faced the same issue with nodetool upgradesstables not converting
all SStables. To convert them all, you can run "nodetool upgradesstables -a" and then do a
check to see which sstables are still in old format. In your /cassandra/data/ directory use
the find command to search for older format sstables named as -la- . This command should work
"find . -name "la*" | grep -v snapshots".

Then run "nodetool upgradesstables keyspace_name" for each of the keyspaces listed. For me,
primarily the system columnfamilies were left in older format.

And about that connection request exceptions, they could be caused if the connecting client
is on an older incompatible java driver. Upgrade the java driver to be compatible with your
Cassandra version. Hope that helps.

Thanks,
Meg Mara 

-----Original Message-----
From: Eric Yablonowitz [mailto:ericy@jana.com] 
Sent: Tuesday, March 28, 2017 9:48 PM
To: user@cassandra.apache.org
Subject: 2.1 to 3.0 upgrade problems

We just upgraded our 33 node cluster from 2.0.10 to 2.1.15 to 3.0.12.  The upgrade from 2.0.10
to 2.1.15 went very smoothly - both the rolling software update and the subsequent ‘nodetool
upgradesstables’.  However we have had a number of issues with the 3.0.12 upgrade:

The first issue we noticed (well into the rolling upgrade) was that the schema versions for
the upgraded nodes were changing with each new node added.  Perhaps related to https://issues.apache.org/jira/browse/CASSANDRA-13274
?  This seemed innocuous until we noticed that commit logs seemed to be growing without bound.
 We attempted to force replay of commit logs by restarting an upgraded node.  However upon
restart we hit:

ERROR [main] 2017-03-27 19:22:38,929 CommitLogReplayer.java:677 - Ignoring commit log replay
error
org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: Unexpected error
deserializing mutation; saved to /tmp/mutation1621984759842154734dat.  This may be caused
by replaying a mutation against a table with the same name but incompatible schema.   Exception
follows: java.io.IOError: java.io.IOException: Corrupt empty row found in unfiltered partition

We were up against the wall with disks filling and had to get nodes restarted or face an outage
so so we used the cassandra.commitlog.ignorereplayerrors=true JVM option to get Cassandra
nodes restarted.  Once we had all nodes upgraded, the schema version stabilized and we stopped
seeing any issues with commitlog replay.

However we are still getting a small number of these seemingly related errors (3-4 per hour):

ERROR [SharedPool-Worker-3] 2017-03-28 23:50:33,998 Message.java:621 - Unexpected exception
during request; channel = [id: 0x2366241f, L:/10.7.150.165:9042 - R:/10.179.229.62:32119]
java.io.IOError: java.io.IOException: Corrupt empty row found in unfiltered partition
	at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer$1.computeNext(UnfilteredRowIteratorSerializer.java:222)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer$1.computeNext(UnfilteredRowIteratorSerializer.java:210)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:129) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.FilteredRows.isEmpty(FilteredRows.java:50) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.Filter.closeIfEmpty(Filter.java:73) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.Filter.applyToPartition(Filter.java:43) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.Filter.applyToPartition(Filter.java:26) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.transform.BasePartitions.hasNext(BasePartitions.java:96) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:707)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:400)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:353)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:227)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:76)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) ~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) [apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) [apache-cassandra-3.0.12.jar:3.0.12]
	at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
[netty-all-4.0.44.Final.jar:4.0.44.Final]
	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357)
[netty-all-4.0.44.Final.jar:4.0.44.Final]
	at io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35)
[netty-all-4.0.44.Final.jar:4.0.44.Final]
	at io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348)
[netty-all-4.0.44.Final.jar:4.0.44.Final]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
	at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [apache-cassandra-3.0.12.jar:3.0.12]
	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45] Caused by: java.io.IOException: Corrupt
empty row found in unfiltered partition
	at org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:382)
~[apache-cassandra-3.0.12.jar:3.0.12]
	at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer$1.computeNext(UnfilteredRowIteratorSerializer.java:217)
~[apache-cassandra-3.0.12.jar:3.0.12]
	... 27 common frames omitted



Next we tried to run ’nodetool upgradetables’.  The vast majority of tables are upgraded
ok, but for a very small minority the nodetool command died with the below (nothing in system.log):

java.lang.AssertionError
	at org.apache.cassandra.db.rows.Rows.collectStats(Rows.java:70)
	at org.apache.cassandra.io.sstable.format.big.BigTableWriter$StatsCollector.applyToRow(BigTableWriter.java:197)
	at org.apache.cassandra.db.transform.BaseRows.applyOne(BaseRows.java:116)
	at org.apache.cassandra.db.transform.BaseRows.add(BaseRows.java:107)
	at org.apache.cassandra.db.transform.UnfilteredRows.add(UnfilteredRows.java:41)
	at org.apache.cassandra.db.transform.Transformation.add(Transformation.java:156)
	at org.apache.cassandra.db.transform.Transformation.apply(Transformation.java:122)
	at org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:147)
	at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:125)
	at org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter.realAppend(DefaultCompactionWriter.java:57)
	at org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:109)
	at org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:195)
	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
	at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:89)
	at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:61)
	at org.apache.cassandra.db.compaction.CompactionManager$5.execute(CompactionManager.java:415)
	at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:307)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
	at java.lang.
Thread.run(Thread.java:745)

Could this be related to https://issues.apache.org/jira/browse/CASSANDRA-13320 ?

Right now it seems like we are stuck with 2.1 tables for a very small minority of our tables.
 I’m not sure what the implications of that are or of the small number of “Corrupt empty
row” errors we are seeing in system.log.  Would appreciate advice.  Thanks!
Mime
View raw message