[
https://issues.apache.org/jira/browse/CASSANDRA-5761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13718133#comment-13718133
]
Andriy Yevsyukov commented on CASSANDRA-5761:
---------------------------------------------
Ok, we had no succes to reproduce this on test application, so I will try to
explain our system a bit. We have 4 cassandra nodes, here our yaml:
{noformat}
cluster_name: 'Our Cassandra Cluster'
num_tokens: 256
initial_token:
max_hint_window_in_ms: 10800000 # 3 hours
hinted_handoff_throttle_in_kb: 1024
max_hints_delivery_threads: 2
authenticator: org.apache.cassandra.auth.AllowAllAuthenticator
authorizer: org.apache.cassandra.auth.AllowAllAuthorizer
permissions_validity_in_ms: 2000
partitioner: org.apache.cassandra.dht.Murmur3Partitioner
commitlog_directory: /commit/cassandra/commitlog
disk_failure_policy: stop
key_cache_size_in_mb:
key_cache_save_period: 14400
row_cache_size_in_mb: 0
row_cache_save_period: 0
row_cache_provider: SerializingCacheProvider
saved_caches_directory: /commit/cassandra/saved_caches
commitlog_sync: periodic
commitlog_sync_period_in_ms: 10000
commitlog_segment_size_in_mb: 32
seed_provider:
- seeds: "our secret seed ip"
flush_largest_memtables_at: 0.75
reduce_cache_sizes_at: 0.85
reduce_cache_capacity_to: 0.6
concurrent_reads: 64
concurrent_writes: 128
memtable_total_space_in_mb: 1024
memtable_flush_writers: 2
memtable_flush_queue_size: 24
trickle_fsync: false
trickle_fsync_interval_in_kb: 10240
storage_port: 7000
ssl_storage_port: 7001
listen_address: 192.168.10.6
native_transport_port: 9042
start_rpc: true
rpc_address:
rpc_port: 9160
rpc_keepalive: true
rpc_server_type: sync
thrift_framed_transport_size_in_mb: 15
thrift_max_message_length_in_mb: 16
incremental_backups: false
snapshot_before_compaction: false
auto_snapshot: true
column_index_size_in_kb: 64
in_memory_compaction_limit_in_mb: 64
multithreaded_compaction: false
compaction_throughput_mb_per_sec: 16
compaction_preheat_key_cache: true
read_request_timeout_in_ms: 60000
range_request_timeout_in_ms: 60000
write_request_timeout_in_ms: 60000
truncate_request_timeout_in_ms: 60000
request_timeout_in_ms: 60000
cross_node_timeout: false
endpoint_snitch: SimpleSnitch
dynamic_snitch_update_interval_in_ms: 100
dynamic_snitch_reset_interval_in_ms: 600000
dynamic_snitch_badness_threshold: 0.1
request_scheduler: org.apache.cassandra.scheduler.NoScheduler
index_interval: 128
server_encryption_options:
internode_encryption: none
keystore: conf/.keystore
keystore_password: cassandra
truststore: conf/.truststore
truststore_password: cassandra
client_encryption_options:
enabled: false
keystore: conf/.keystore
keystore_password: cassandra
internode_compression: all
inter_dc_tcp_nodelay: true
{noformat}
As I said we use CQL3 to create our tables structure before we start, here
sample of how we do that for one CF:
{noformat}
CREATE KEYSPACE common WITH replication = {'class': 'SimpleStrategy',
'replication_factor': 1};
CREATE TABLE events(
key varchar,
event_id bigint,
rotation_number bigint,
source_name varchar,
event_date bigint,
event_day bigint,
timestamp bigint,
description varchar,
type varchar,
league varchar,
points float,
units varchar,
has_result boolean,
result varchar,
result_desc varchar,
result_score varchar,
actual boolean,
PRIMARY KEY (key)
)WITH COMPACT STORAGE;
CREATE INDEX ON events (event_id);
CREATE INDEX ON events (rotation_number);
CREATE INDEX ON events (source_name);
CREATE INDEX ON events (event_date);
CREATE INDEX ON events (event_day);
CREATE INDEX ON events (type);
CREATE INDEX ON events (league);
CREATE INDEX ON events (common_total_units);
CREATE INDEX ON events (has_result);
CREATE INDEX ON events (actual);
{noformat}
We use our own resource adapter to work with Cassandra and our own api over
Hector. Here the fragment how we store data into CF:
{noformat}
class RowMutationBatchContextImpl<K,N> extends AbstractBuilderContext<K,N>
implements RowMutationBatchContext<K,N> {
private K key;
private Mutator mutator;
private RowMutation<K, N> rowMutation;
private MutationBatch mutationBatch;
RowMutationBatchContextImpl(Mutator mutator) {
this.mutator = mutator;
}
@Override
public void setKey(K key) {
this.key = key;
}
@Override
public void setRowMutation(RowMutation<K, N> rowMutation){
this.rowMutation = rowMutation;
}
@Override
public RowMutation<K,N> getRowMutation() {
return rowMutation;
}
@Override
public void setMutationBatch(MutationBatch mutationBatch) {
this.mutationBatch = mutationBatch;
}
@Override
public MutationBatch getMutationBatch() {
return mutationBatch;
}
@Override
public <V> void insert(ColumnMetaData<N, V> columnDef, V value) {
mutator.addInsertion(toKeyComponents(key),
getColumnFamilyMetaData().getName(), createHColumn(columnDef, value));
}
@Override
public <M,V> void insertDynamic(ColumnMetaData<M, V> columnDef, V value) {
mutator.addInsertion(toKeyComponents(key),
getColumnFamilyMetaData().getName(), createHColumn(columnDef, value));
}
@Override
public void removeColumns(Collection<ColumnMetaData<N, ?>>
columnDefinitions) {
for (ColumnMetaData<N,?> columnDef : columnDefinitions){
removeColumn(columnDef);
}
}
@Override
public void removeColumn(ColumnMetaData<N, ?> columnDef) {
mutator.addDeletion(toKeyComponents(key),
getColumnFamilyMetaData().getName(),
getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
ByteBufferSerializer.get());
}
@Override
public void removeAll() {
mutator.addDeletion(toKeyComponents(key),
getColumnFamilyMetaData().getName());
}
@Override
public void execute(){
mutator.execute();
}
}
{noformat}
This is how we do indexed query (I know you deprecated IndexedSlicesQuery, but
for some reason RangeSlicesQuery doesn't work for us):
{noformat}
public class IndexSlicesContextImpl<K, N> extends AbstractSliceSetContext<K,N>
implements IndexedSlicesContext<K, N> {
private K startKey;
private IndexedSlicesQuery hQuery;
public IndexSlicesContextImpl(IndexedSlicesQuery hQuery) {
this.hQuery = hQuery;
}
@Override
public void setStartKey(K key) {
this.startKey = key;
}
private Object getStartKey(){
if (startKey == null){
return null;
}
return toKeyComponents(startKey);
}
@Override
public void setResultCount(int count) {
hQuery.setRowCount(count);
}
@Override
public <V> void addEqualToExpression(ColumnMetaData<N, V> columnDef, V
value) {
hQuery.addEqualsExpression(getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
getTypeInstance(columnDef.getValueClass()).toComponents(value));
}
@Override
public <V> void addGreaterThenExpression(ColumnMetaData<N, V> columnDef, V
value) {
hQuery.addGtExpression(getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
getTypeInstance(columnDef.getValueClass()).toComponents(value));
}
@Override
public <V> void addGreaterThenEqualExpression(ColumnMetaData<N, V>
columnDef, V value) {
hQuery.addGteExpression(getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
getTypeInstance(columnDef.getValueClass()).toComponents(value));
}
@Override
public <V> void addLessThenExpression(ColumnMetaData<N, V> columnDef, V
value) {
hQuery.addLtExpression(getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
getTypeInstance(columnDef.getValueClass()).toComponents(value));
}
@Override
public <V> void addLessThenEqualExpression(ColumnMetaData<N, V> columnDef,
V value) {
hQuery.addLteExpression(getTypeInstance(columnDef.getNameClass()).toComponents(columnDef.getName()),
getTypeInstance(columnDef.getValueClass()).toComponents(value));
}
@Override
protected Rows getResultRows() {
hQuery.setColumnFamily(getColumnFamilyMetaData().getName());
hQuery.setRange(getStartName(), getFinishName(), isReversed(),
getCount());
Object[] names = getColumnNames();
if (names != null) {
hQuery.setColumnNames(names);
}
hQuery.setStartKey(getStartKey());
return (Rows) hQuery.execute().get();
}
}
{noformat}
We have many similar CFs but for different purposes - but the structure pretty
much the same - about 15-20 permanent columns and from 10 to 30 (or even more)
dynamic columns (we don't use dynamic columns for index). As you can see, our
key column is String type.
We usually used 28 chars key - first part of the key is event_id and the second
is generated string numbers - but the length was 28. The problem started when
for some event_id length has changed so the key length became 38. When we found
who changed the event_id length and fixed that the problem has gone. But what
if in the future we will change the length again?
Thanks.
> Issue with secondary index sstable.
> -----------------------------------
>
> Key: CASSANDRA-5761
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5761
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 1.2.5
> Reporter: Andriy Yevsyukov
> Priority: Critical
>
> With Cassandra 1.2.5 having issue very similar to
> [CASSANDRA-5225|https://issues.apache.org/jira/browse/CASSANDRA-5225] but for
> secondary index sstable. Every query that uses this index fails in Hector
> with ConnectionTimeout but cassandra log says that reason is:
> {noformat}
> ERROR [ReadStage:55803] 2013-07-15 12:11:35,392 CassandraDaemon.java (line
> 175) Exception in thread Thread[ReadStage:55803,5,main]
> java.lang.RuntimeException:
> org.apache.cassandra.io.sstable.CorruptSSTableException:
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: invalid
> column name length 0
> (/data/cassandra/data/betting/events/betting-events.events_sport_type_idx-ic-1-Data.db,
> 19658 bytes remaining)
> at
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1582)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException:
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: invalid
> column name length 0
> (/data/cassandra/data/betting/events/betting-events.events_sport_type_idx-ic-1-Data.db,
> 19658 bytes remaining)
> at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:108)
> at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:39)
> at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:90)
> at
> org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:171)
> at
> org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:154)
> at
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:143)
> at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:86)
> at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:45)
> at
> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:134)
> at
> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:84)
> at
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:293)
> at
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:65)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1357)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1214)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1126)
> at
> org.apache.cassandra.db.index.keys.KeysSearcher$1.computeNext(KeysSearcher.java:140)
> at
> org.apache.cassandra.db.index.keys.KeysSearcher$1.computeNext(KeysSearcher.java:109)
> at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
> at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
> at
> org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1466)
> at
> org.apache.cassandra.db.index.keys.KeysSearcher.search(KeysSearcher.java:82)
> at
> org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:548)
> at
> org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:1454)
> at
> org.apache.cassandra.service.RangeSliceVerbHandler.executeLocally(RangeSliceVerbHandler.java:44)
> at
> org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1076)
> at
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1578)
> ... 3 more
> Caused by: org.apache.cassandra.db.ColumnSerializer$CorruptColumnException:
> invalid column name length 0
> (/data/cassandra/data/betting/events/betting-events.events_sport_type_idx-ic-1-Data.db,
> 19658 bytes remaining)
> at
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException.create(ColumnSerializer.java:148)
> at
> org.apache.cassandra.db.RangeTombstone$Serializer.deserializeBody(RangeTombstone.java:279)
> at
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:90)
> at
> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:73)
> at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:104)
> ... 30 more
> {noformat}
> We cleared CF but we have a lot of insertion and in few hours the issue
> happened again. Are there any advices?
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira