[
https://issues.apache.org/jira/browse/CASSANDRA-13004?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15855257#comment-15855257
]
Nimi Wariboko Jr. commented on CASSANDRA-13004:
-----------------------------------------------
Hi,
I think I'm hitting the same issue on 3.5 - for us it prevents us from
bootstrapping any new nodes. Oddly we can sometimes do reads just fine (SELECT
* FROM table), but other times we get a timeout exception with the "Corrupt
length" in our logs. We tried scrub (no effect), and we tried dumping the
entire table to JSON, TRUNCATING it, and reinserting all the data - and even
the new fresh sstables hit this issue.
(Node is running 3.5, but we are using 3.10's sstabledump).
{code}
db@cass2:~/apache-cassandra-3.10/bin$ sudo ../tools/bin/sstabledump
/mnt/dsk2/var/lib/cassandra/data/cmuser/users-68b04ac07c5011e5a85daf9a47bddf3d/ma-9473-big-Data.db
WARN 03:41:14,442 Only 30.140GiB free across all data volumes. Consider adding
more capacity to your cluster or removing obsolete snapshots
[
{
"partition" : {
"key" : [ "560c3b78-adf2-11e6-8ef2-f45c89b76d37" ],
"position" : 0
},
"rows" : [ ]
}
]Exception in thread "main"
org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted:
/mnt/dsk2/var/lib/cassandra/data/cmuser/users-68b04ac07c5011e5a85daf9a47bddf3d/ma-9473-big-Data.db
at
org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:134)
at
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:100)
at
org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.tools.JsonTransformer.serializePartition(JsonTransformer.java:206)
at
java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:184)
at
java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:175)
at java.util.Iterator.forEachRemaining(Iterator.java:116)
at
java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
at
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
at
java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:151)
at
java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:174)
at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
at
java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:418)
at
org.apache.cassandra.tools.JsonTransformer.toJson(JsonTransformer.java:99)
at org.apache.cassandra.tools.SSTableExport.main(SSTableExport.java:240)
Caused by: java.io.IOException: Corrupt (negative) value length encountered
at
org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:425)
at
org.apache.cassandra.db.rows.Cell$Serializer.deserialize(Cell.java:245)
at
org.apache.cassandra.db.rows.UnfilteredSerializer.readComplexColumn(UnfilteredSerializer.java:636)
at
org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$deserializeRowBody$1(UnfilteredSerializer.java:577)
at org.apache.cassandra.utils.btree.BTree.applyForwards(BTree.java:1222)
at org.apache.cassandra.utils.btree.BTree.applyForwards(BTree.java:1226)
at org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1177)
at org.apache.cassandra.db.Columns.apply(Columns.java:377)
at
org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:571)
at
org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:440)
at
org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:95)
at
org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:73)
at
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
at
org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:122)
... 16 more
db@cass2:~/apache-cassandra-3.10/bin$
{code}
We have done ALTER TABLE's on this table, but not for a couple months.
Similarly, I notice we both are using collection types with custom types -
could that have something to do with the issue?
{code}
CREATE TABLE cmuser.users (
id timeuuid PRIMARY KEY,
account_id text,
api_key text,
avatar text,
channel text,
city text,
company text,
country text,
created timestamp,
custom_id text,
customer text,
date_of_birth timestamp,
description text,
disabled boolean,
email text,
hidden_from_community boolean,
language text,
last_action timestamp,
monitor boolean,
name text,
notes text,
objectid text,
organizationid text,
orgcontract set<frozen<contract>>,
orgcontracts2 map<timeuuid, frozen<contract>>,
orgispaid boolean,
orglevel int,
orgownership set<frozen<ownership_rule>>,
orgpayout double,
orgreferrer timeuuid,
orgrevshare revshare,
orgsubnetwork timeuuid,
orgtags set<text>,
owner_key text,
parentemail text,
password blob,
sh_providerno bigint,
social map<text, text>,
state text,
test_account_id text,
test_customer text
)
cqlsh:cmuser> describe type revshare
CREATE TYPE cmuser.revshare (
partner decimal,
subnetwork decimal,
commission decimal,
ugc decimal,
version int,
direct decimal,
fanfunded decimal,
bonus_partner decimal,
cpm decimal,
capcpm decimal
);
cqlsh:cmuser> describe type contract
CREATE TYPE cmuser.contract (
id timeuuid,
provider text,
templateid text,
templatename text,
documentid text,
downloadlink text,
starts timestamp,
ends timestamp,
signed boolean,
signedon timestamp,
sessionid text,
sessionexpire timestamp
);
cqlsh:cmuser> describe type ownership_rule
CREATE TYPE cmuser.ownership_rule (
field text,
value text,
readonly boolean,
hiderevenue boolean,
isprimary boolean,
exclude boolean,
skippayout boolean,
orgpayout double,
revshare frozen<revshare>
);
{code}
> Corruption while adding a column to a table
> -------------------------------------------
>
> Key: CASSANDRA-13004
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13004
> Project: Cassandra
> Issue Type: Bug
> Reporter: Stanislav Vishnevskiy
>
> We had the following schema in production.
> {code:none}
> CREATE TYPE IF NOT EXISTS discord_channels.channel_recipient (
> nick text
> );
> CREATE TYPE IF NOT EXISTS discord_channels.channel_permission_overwrite (
> id bigint,
> type int,
> allow_ int,
> deny int
> );
> CREATE TABLE IF NOT EXISTS discord_channels.channels (
> id bigint,
> guild_id bigint,
> type tinyint,
> name text,
> topic text,
> position int,
> owner_id bigint,
> icon_hash text,
> recipients map<bigint, frozen<channel_recipient>>,
> permission_overwrites map<bigint, frozen<channel_permission_overwrite>>,
> bitrate int,
> user_limit int,
> last_pin_timestamp timestamp,
> last_message_id bigint,
> PRIMARY KEY (id)
> );
> {code}
> And then we executed the following alter.
> {code:none}
> ALTER TABLE discord_channels.channels ADD application_id bigint;
> {code}
> And one row (that we can tell) got corrupted at the same time and could no
> longer be read from the Python driver.
> {code:none}
> [E 161206 01:56:58 geventreactor:141] Error decoding response from Cassandra.
> ver(4); flags(0000); stream(27); op(8); offset(9); len(887); buffer:
> '\x84\x00\x00\x1b\x08\x00\x00\x03w\x00\x00\x00\x02\x00\x00\x00\x01\x00\x00\x00\x0f\x00\x10discord_channels\x00\x08channels\x00\x02id\x00\x02\x00\x0eapplication_id\x00\x02\x00\x07bitrate\x00\t\x00\x08guild_id\x00\x02\x00\ticon_hash\x00\r\x00\x0flast_message_id\x00\x02\x00\x12last_pin_timestamp\x00\x0b\x00\x04name\x00\r\x00\x08owner_id\x00\x02\x00\x15permission_overwrites\x00!\x00\x02\x000\x00\x10discord_channels\x00\x1cchannel_permission_overwrite\x00\x04\x00\x02id\x00\x02\x00\x04type\x00\t\x00\x06allow_\x00\t\x00\x04deny\x00\t\x00\x08position\x00\t\x00\nrecipients\x00!\x00\x02\x000\x00\x10discord_channels\x00\x11channel_recipient\x00\x01\x00\x04nick\x00\r\x00\x05topic\x00\r\x00\x04type\x00\x14\x00\nuser_limit\x00\t\x00\x00\x00\x01\x00\x00\x00\x08\x03\x8a\x19\x8e\xf8\x82\x00\x01\xff\xff\xff\xff\x00\x00\x00\x04\x00\x00\xfa\x00\x00\x00\x00\x08\x00\x00\xfa\x00\x00\xf8G\xc5\x00\x00\x00\x00\x00\x00\x00\x08\x03\x8b\xc0\xb5nB\x00\x02\x00\x00\x00\x08G\xc5\xffI\x98\xc4\xb4(\x00\x00\x00\x03\x8b\xc0\xa8\xff\xff\xff\xff\x00\x00\x01<\x00\x00\x00\x06\x00\x00\x00\x08\x03\x81L\xea\xfc\x82\x00\n\x00\x00\x00$\x00\x00\x00\x08\x03\x81L\xea\xfc\x82\x00\n\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x08\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x08\x03\x8a\x1e\xe6\x8b\x80\x00\n\x00\x00\x00$\x00\x00\x00\x08\x03\x8a\x1e\xe6\x8b\x80\x00\n\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x040\x07\xf8Q\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x08\x03\x8a\x1f\x1b{\x82\x00\x00\x00\x00\x00$\x00\x00\x00\x08\x03\x8a\x1f\x1b{\x82\x00\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x04\x00\x07\xf8Q\x00\x00\x00\x04\x10\x00\x00\x00\x00\x00\x00\x08\x03\x8a\x1fH6\x82\x00\x01\x00\x00\x00$\x00\x00\x00\x08\x03\x8a\x1fH6\x82\x00\x01\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x04\x00\x05\xe8A\x00\x00\x00\x04\x10\x02\x00\x00\x00\x00\x00\x08\x03\x8a+=\xca\xc0\x00\n\x00\x00\x00$\x00\x00\x00\x08\x03\x8a+=\xca\xc0\x00\n\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x04\x00\x00\x08\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x08\x03\x8a\x8f\x979\x80\x00\n\x00\x00\x00$\x00\x00\x00\x08\x03\x8a\x8f\x979\x80\x00\n\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x04\x00
>
> \x08\x01\x00\x00\x00\x04\xc4\xb4(\x00\xff\xff\xff\xff\x00\x00\x00O[f\x80Q\x07general\x05\xf8G\xc5\xffI\x98\xc4\xb4(\x00\xf8O[f\x80Q\x00\x00\x00\x02\x04\xf8O[f\x80Q\x00\xf8G\xc5\xffI\x98\x01\x00\x00\xf8O[f\x80Q\x00\x00\x00\x00\xf8G\xc5\xffI\x97\xc4\xb4(\x06\x00\xf8O\x7fe\x1fm\x08\x03\x00\x00\x00\x01\x00\x00\x00\x00\x04\x00\x00\x00\x00'
> {code}
> And then in cqlsh when trying to read the row we got this.
> {code:none}
> /usr/bin/cqlsh.py:632: DateOverFlowWarning: Some timestamps are larger than
> Python datetime can represent. Timestamps are displayed in milliseconds from
> epoch.
> Traceback (most recent call last):
> File "/usr/bin/cqlsh.py", line 1301, in perform_simple_statement
> result = future.result()
> File
> "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.5.0.post0-d8d0456.zip/cassandra-driver-3.5.0.post0-d8d0456/cassandra/cluster.py",
> line 3650, in result
> raise self._final_exception
> UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 2:
> invalid start byte
> {code}
> We tried to read the data and it would refuse to read the name column (the
> UTF8 error) and the last_pin_timestamp column had an absurdly large value.
> We ended up rewriting the whole row as we had the data in another place and
> it fixed the problem. However there is clearly a race condition in the schema
> change sub-system.
> Any ideas?
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)