[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-10-21 Thread Jonathan Ellis (Updated) (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis updated CASSANDRA-2810:
--

Fix Version/s: (was: 1.0.1)

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
>Assignee: Brandon Williams
> Fix For: 0.8.7
>
> Attachments: 2810-v2.txt, 2810-v3.txt, 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
>   at 
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map.collect(PigMapOnly.java:48)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:239)
>   at 
> org.apache.pig.backend.hadoop.executione

[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-10-21 Thread Jonathan Ellis (Updated) (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jonathan Ellis updated CASSANDRA-2810:
--

Fix Version/s: 1.0.1

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
>Assignee: Brandon Williams
> Fix For: 0.8.7
>
> Attachments: 2810-v2.txt, 2810-v3.txt, 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
>   at 
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map.collect(PigMapOnly.java:48)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:239)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapRe

[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-08-24 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-2810:


Reviewer: jeromatron

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
>Assignee: Brandon Williams
> Attachments: 2810-v2.txt, 2810-v3.txt, 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
>   at 
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map.collect(PigMapOnly.java:48)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:239)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMap

[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-08-23 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-2810:


Attachment: 2810-v3.txt

v3 also removes decomposing the values before inserting and instead forces them 
into a ByteBuffer with objToBB, since we actually don't care about the type. 
(why did we ever change this?)

This means that a UDF that doesn't preserve the schema and hands us back 
DataByteArrays when we fed it specific types can't make us fail anymore.

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
>Assignee: Brandon Williams
> Attachments: 2810-v2.txt, 2810-v3.txt, 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
>   at 
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.

[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-08-09 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-2810:


Attachment: 2810-v2.txt

It looks like the final problem here is that IntegerType always returns a 
BigInteger, which pig does not like.  This is unfortunate since IntegerType 
can't be easily subclassed and overridden to return ints.

v2 instead adds a setTupleValue method that is always used for adding values to 
tuples, and houses all the special-casing currently needed and provides a spot 
for more in the future, rather than proliferating custom type converters since 
I'm sure IntegerType won't be alone here.

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
>Assignee: Brandon Williams
> Attachments: 2810-v2.txt, 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$

[jira] [Updated] (CASSANDRA-2810) RuntimeException in Pig when using "dump" command on column name

2011-06-24 Thread Brandon Williams (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-2810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Brandon Williams updated CASSANDRA-2810:


Attachment: 2810.txt

Patch to use a custom AbstractType in place of BytesType to nip this in the 
bud, rather than have a bunch of one-off checks.  Also fixes a bug where the 
supercolumn name is never set.

> RuntimeException in Pig when using "dump" command on column name
> 
>
> Key: CASSANDRA-2810
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2810
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 0.8.1
> Environment: Ubuntu 10.10, 32 bits
> java version "1.6.0_24"
> Brisk beta-2 installed from Debian packages
>Reporter: Silvère Lestang
> Attachments: 2810.txt
>
>
> This bug was previously report on [Brisk bug 
> tracker|https://datastax.jira.com/browse/BRISK-232].
> In cassandra-cli:
> {code}
> [default@unknown] create keyspace Test
> with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
> and strategy_options = [{replication_factor:1}];
> [default@unknown] use Test;
> Authenticated to keyspace: Test
> [default@Test] create column family test;
> [default@Test] set test[ascii('row1')][long(1)]=integer(35);
> set test[ascii('row1')][long(2)]=integer(36);
> set test[ascii('row1')][long(3)]=integer(38);
> set test[ascii('row2')][long(1)]=integer(45);
> set test[ascii('row2')][long(2)]=integer(42);
> set test[ascii('row2')][long(3)]=integer(33);
> [default@Test] list test;
> Using default limit of 100
> ---
> RowKey: 726f7731
> => (column=0001, value=35, timestamp=1308744931122000)
> => (column=0002, value=36, timestamp=1308744931124000)
> => (column=0003, value=38, timestamp=1308744931125000)
> ---
> RowKey: 726f7732
> => (column=0001, value=45, timestamp=1308744931127000)
> => (column=0002, value=42, timestamp=1308744931128000)
> => (column=0003, value=33, timestamp=1308744932722000)
> 2 Rows Returned.
> [default@Test] describe keyspace;
> Keyspace: Test:
>   Replication Strategy: org.apache.cassandra.locator.SimpleStrategy
>   Durable Writes: true
> Options: [replication_factor:1]
>   Column Families:
> ColumnFamily: test
>   Key Validation Class: org.apache.cassandra.db.marshal.BytesType
>   Default column value validator: 
> org.apache.cassandra.db.marshal.BytesType
>   Columns sorted by: org.apache.cassandra.db.marshal.BytesType
>   Row cache size / save period in seconds: 0.0/0
>   Key cache size / save period in seconds: 20.0/14400
>   Memtable thresholds: 0.571875/122/1440 (millions of ops/MB/minutes)
>   GC grace seconds: 864000
>   Compaction min/max thresholds: 4/32
>   Read repair chance: 1.0
>   Replicate on write: false
>   Built indexes: []
> {code}
> In Pig command line:
> {code}
> grunt> test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS 
> (rowkey:chararray, columns: bag {T: (name:long, value:int)});
> grunt> value_test = foreach test generate rowkey, columns.name, columns.value;
> grunt> dump value_test;
> {code}
> In /var/log/cassandra/system.log, I have severals time this exception:
> {code}
> INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 
> TaskInProgress.java (line 551) Error from 
> attempt_201106210955_0051_m_00_3: java.lang.RuntimeException: Unexpected 
> data type -1 found in stream.
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:478)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeBag(BinInterSedes.java:522)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:361)
>   at org.apache.pig.data.BinInterSedes.writeTuple(BinInterSedes.java:541)
>   at org.apache.pig.data.BinInterSedes.writeDatum(BinInterSedes.java:357)
>   at 
> org.apache.pig.impl.io.InterRecordWriter.write(InterRecordWriter.java:73)
>   at org.apache.pig.impl.io.InterStorage.putNext(InterStorage.java:87)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:138)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat$PigRecordWriter.write(PigOutputFormat.java:97)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.write(MapTask.java:638)
>   at 
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map.collect(PigMapOnly.java:48)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipel