incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From POUGET Laurent <laurent.pou...@carboatmedia.fr>
Subject RE: SStable Writer and composite key
Date Thu, 31 Jan 2013 14:52:44 GMT
Ok, in order to solve : java.lang.AssertionError: Added column does not sort as the last column

I just have to make a scrub with nodetools.

Is there a programmatic way to scrub and compact new sstable on an active node ?

Thx,

Laurent Pouget


De : POUGET Laurent [mailto:laurent.pouget@carboatmedia.fr]
Envoyé : jeudi 31 janvier 2013 11:31
À : user@cassandra.apache.org
Objet : RE: SStable Writer and composite key

Thanks,

Import is ok now but :

·         If I query my table with 1.2.0, I have an exception :


java.lang.AssertionError: Added column does not sort as the last column
       at org.apache.cassandra.db.ArrayBackedSortedColumns.addColumn(ArrayBackedSortedColumns.java:131)
       at org.apache.cassandra.db.AbstractColumnContainer.addColumn(AbstractColumnContainer.java:109)
       at org.apache.cassandra.db.AbstractColumnContainer.addColumn(AbstractColumnContainer.java:104)
       at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:171)
       at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:136)
       at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:84)
       at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:286)
       at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:61)
       at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1362)
       at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1222)
       at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1134)
       at org.apache.cassandra.db.Table.getRow(Table.java:348)
       at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:70)
       at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1048)
       at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1506)
       at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
       at java.lang.Thread.run(Unknown Source)

With 1.2.1 data doesn't load :

ERROR  Corrupt sstable <path>/<keyspace_name>/<cf_name>/<cf_name>_logs-ib-42=[TOC.txt,
Summary.db, Index
.db, Data.db, Filter.db, Digest.sha1, Statistics.db]; skipped
java.io.EOFException
        at java.io.DataInputStream.readFully(Unknown Source)
        at java.io.DataInputStream.readUTF(Unknown Source)
        at java.io.DataInputStream.readUTF(Unknown Source)
        at org.apache.cassandra.io.sstable.SSTableMetadata$SSTableMetadataSerializer.deserialize(SSTableMetadata.java:27
8)
        at org.apache.cassandra.io.sstable.SSTableMetadata$SSTableMetadataSerializer.deserialize(SSTableMetadata.java:25
1)
        at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:166)
        at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:149)
        at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:238)
        at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
        at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
        at java.util.concurrent.FutureTask.run(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
        at java.lang.Thread.run(Unknown Source)


Now with your advice, my java code looks like :

...

simpleUnsortedWriter.newRow(bytes(id));

ByteBuffer date = bytes(_date);
ByteBuffer request = bytes(_request);

long timestamp = System.currentTimeMillis() * 1000;

addColumn(date, request, "", "", timestamp);
addColumn(date, request, "data1", data1, timestamp);
addColumn(date, request, "data2", data2, timestamp);

...

public void addColumn(ByteBuffer date,
                    ByteBuffer request, String field, String value, long timestamp)
                    throws IOException {
       builder = compositeType.builder();

       builder.add(date);
       builder.add(request);
       builder.add(bytes(field));

       simpleUnsortedWriter.addColumn(builder.build(),
                    bytes(value), timestamp);
}





De : aaron morton [mailto:aaron@thelastpickle.com]
Envoyé : jeudi 31 janvier 2013 00:03
À : user@cassandra.apache.org<mailto:user@cassandra.apache.org>
Objet : Re: SStable Writer and composite key

This is what a row of your table will look like internally...

-------------------
RowKey: id-value
=> (column=date-value:request-value:, value=, timestamp=1359586739456000)
=> (column=date-value:request-value:data1, value=64617461312d76616c7565, timestamp=1359586739456000)
=> (column=date-value:request-value:data2, value=64617461322d76616c7565, timestamp=1359586739456000)

where id-value is the value of id column, and date-value is the....

So you need to construct triples for each column value.
The first col name is the (date-value, request-value, empty)
The second col name is (date-value, request-value, "data1") and has the value of data1.

Hope that helps.

-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 30/01/2013, at 3:11 AM, POUGET Laurent <laurent.pouget@carboatmedia.fr<mailto:laurent.pouget@carboatmedia.fr>>
wrote:

Hi,


I have some trouble to request my data. I use SSTableSimpleUnsortedWriter to write SSTable.
Writing and Importing works fine.
I think, I'm misusing CompositeType.Builder with SSTableSimpleUnsortedWriter.
Do you have any idea ?

Thanks

Here is my case :

/**
* CREATE STATEMENT
*/

CREATE TABLE raw_data (
  id text,
  date text,
  request text,
  data1 text,
  data2 text,
  PRIMARY KEY (id, date, request)
) WITH
  bloom_filter_fp_chance=0.010000 AND
  caching='KEYS_ONLY' AND
  comment='' AND
  dclocal_read_repair_chance=0.000000 AND
  gc_grace_seconds=864000 AND
  read_repair_chance=0.100000 AND
  replicate_on_write='true' AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={'sstable_compression': 'SnappyCompressor'};

/**
* JAVA CODE
*/

List<AbstractType<?>> compositeList = new ArrayList<AbstractType<?>>();

compositeList.add(UTF8Type.instance);
compositeList.add(UTF8Type.instance);

IPartitioner<?> partitioner = StorageService.getPartitioner();
dir = Directories.create(keyspace.getKeyspaceName(), columnFamily.getName()).getDirectoryForNewSSTables(0);

simpleUnsortedWriter = new SSTableSimpleUnsortedWriter(dir,
                                                               partitioner, keyspace.getKeyspaceName(),
                                                               columnFamily.getName(), UTF8Type.instance,
null,
                                                               32);

CompositeType.Builder builderRequestDate = new CompositeType.Builder( CompositeType.getInstance
                      (compositeList) );
CompositeType.Builder builderUrl = new CompositeType.Builder( CompositeType.getInstance(compositeList)
);

simpleUnsortedWriter.newRow(bytes(id));

builderRequestDate.add(bytes("date"));
builderRequestDate.add(bytes("request"));

long timestamp = System.currentTimeMillis() * 1000;

simpleUnsortedWriter.addColumn(builderRequestDate.build(),
                               bytes(date), timestamp);
simpleUnsortedWriter.addColumn(builderUrl.build(),
                               bytes(request), timestamp);

simpleUnsortedWriter.addColumn(bytes("data1"),
                               bytes(data1), timestamp);
simpleUnsortedWriter.addColumn(bytes("data2"),
                               bytes(data2), timestamp);

simpleUnsortedWriter.close();


________________________________

Laurent Pouget
Ingénieur étude et développement
Tel : 01.84.95.11.20

Car & Boat Media
22 Rue Joubert  75009 Paris







Mime
View raw message