Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-2810

RuntimeException in Pig when using "dump" command on column name

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Fix Version/s: 0.8.7
    • Component/s: None
    • Labels:
      None
    • Environment:

      Ubuntu 10.10, 32 bits
      java version "1.6.0_24"
      Brisk beta-2 installed from Debian packages

      Description

      This bug was previously report on Brisk bug tracker.

      In cassandra-cli:

      [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=0000000000000001, value=35, timestamp=1308744931122000)
      => (column=0000000000000002, value=36, timestamp=1308744931124000)
      => (column=0000000000000003, value=38, timestamp=1308744931125000)
      -------------------
      RowKey: 726f7732
      => (column=0000000000000001, value=45, timestamp=1308744931127000)
      => (column=0000000000000002, value=42, timestamp=1308744931128000)
      => (column=0000000000000003, 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: 200000.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: []
      

      In Pig command line:

      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;
      

      In /var/log/cassandra/system.log, I have severals time this exception:

      INFO [IPC Server handler 3 on 8012] 2011-06-22 15:03:28,533 TaskInProgress.java (line 551) Error from attempt_201106210955_0051_m_000000_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(PigMapBase.java:232)
      	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMapBase.java:53)
      	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
      	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
      	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
      	at org.apache.hadoop.mapred.Child$4.run(Child.java:259)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:396)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
      	at org.apache.hadoop.mapred.Child.main(Child.java:253)
      

      and the request failed.

      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.value;
      
      grunt> dump value_test;
      

      This time, without the column name, it's work (but the value are displayed as char instead of integer). Result:

      (row1,{(#),($),(&)})
      (row2,{(-),(*),(!)})
      

      Now we do the same test but we set a comparator to the CF.

      [default@Test] create column family test with comparator = 'LongType';
      
      [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=1, value=35, timestamp=1308748643506000)
      => (column=2, value=36, timestamp=1308748643508000)
      => (column=3, value=38, timestamp=1308748643509000)
      -------------------
      RowKey: 726f7732
      => (column=1, value=45, timestamp=1308748643510000)
      => (column=2, value=42, timestamp=1308748643512000)
      => (column=3, value=33, timestamp=1308748645138000)
      
      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.LongType
            Row cache size / save period in seconds: 0.0/0
            Key cache size / save period in seconds: 200000.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: []
      
      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;
      

      This time it's work as expected (appart from the value displayed as char). Result:

      (row1,{(1),(2),(3)},{(#),($),(&)})
      (row2,{(1),(2),(3)},{(-),(*),(!)})
      
      1. 2810-v3.txt
        4 kB
        Brandon Williams
      2. 2810-v2.txt
        4 kB
        Brandon Williams
      3. 2810.txt
        5 kB
        Brandon Williams

        Activity

        Hide
        brandon.williams Brandon Williams added a comment -

        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.

        Show
        brandon.williams Brandon Williams added a comment - 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.
        Hide
        jbellis Jonathan Ellis added a comment -

        DataByteArray is some kind of Pig thing?

        Show
        jbellis Jonathan Ellis added a comment - DataByteArray is some kind of Pig thing?
        Hide
        brandon.williams Brandon Williams added a comment -

        Yes, basically a byte array, but it's the pig type.

        Show
        brandon.williams Brandon Williams added a comment - Yes, basically a byte array, but it's the pig type.
        Hide
        silvere Silvère Lestang added a comment -

        I try again after applying 2810.txt and the patch from bug CASSANDRA-2777 and the bug is still here.
        With the patch, you need to replace

        test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS (rowkey:chararray, columns: bag {T: (name:long, value:int)});
        

        by

        test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS ();
        

        because CassandraStorage takes care of the schema.

        I try:

        grunt> describe test;
        test: {key: chararray,columns: {(name: long,value: int)}}
        

        so we can see that the patch from bug 2777 works correctly (I also test with different types for value).
        But when I dump test, I still have the same exception.

        Show
        silvere Silvère Lestang added a comment - I try again after applying 2810.txt and the patch from bug CASSANDRA-2777 and the bug is still here. With the patch, you need to replace test = LOAD 'cassandra: //Test/test' USING CassandraStorage() AS (rowkey:chararray, columns: bag {T: (name: long , value: int )}); by test = LOAD 'cassandra: //Test/test' USING CassandraStorage() AS (); because CassandraStorage takes care of the schema. I try: grunt> describe test; test: {key: chararray,columns: {(name: long ,value: int )}} so we can see that the patch from bug 2777 works correctly (I also test with different types for value). But when I dump test, I still have the same exception.
        Hide
        silvere Silvère Lestang added a comment -

        After more test (with both patches), path 2810.txt doesn't seems to solve the bug.
        Here is a new test case:
        Create a Test keyspace and a test column family with key_validation_class = 'AsciiType' and comparator = 'LongType' and default_validation_class = 'IntegerType' (don't use the cli because of CASSANDRA-2831).
        Insert some data:

        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);
        

        In Pig cli:

        test = LOAD 'cassandra://Test/test' USING CassandraStorage() AS ();
        dump test;
        

        The same exception as before is raised:

         INFO [IPC Server handler 4 on 8012] 2011-06-27 16:40:28,562 TaskInProgress.java (line 551) Error from attempt_201106271436_0012_m_000000_1: 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.map(PigMapBase.java:224)
        	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMapBase.java:53)
        	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
        	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
        	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369)
        	at org.apache.hadoop.mapred.Child$4.run(Child.java:259)
        	at java.security.AccessController.doPrivileged(Native Method)
        	at javax.security.auth.Subject.doAs(Subject.java:396)
        	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
        	at org.apache.hadoop.mapred.Child.main(Child.java:253)
        
        
        Show
        silvere Silvère Lestang added a comment - After more test (with both patches), path 2810.txt doesn't seems to solve the bug. Here is a new test case: Create a Test keyspace and a test column family with key_validation_class = 'AsciiType' and comparator = 'LongType' and default_validation_class = 'IntegerType' (don't use the cli because of CASSANDRA-2831 ). Insert some data: 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); In Pig cli: test = LOAD 'cassandra: //Test/test' USING CassandraStorage() AS (); dump test; The same exception as before is raised: INFO [IPC Server handler 4 on 8012] 2011-06-27 16:40:28,562 TaskInProgress.java (line 551) Error from attempt_201106271436_0012_m_000000_1: 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.map(PigMapBase.java:224) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.map(PigMapBase.java:53) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:369) at org.apache.hadoop.mapred.Child$4.run(Child.java:259) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:396) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059) at org.apache.hadoop.mapred.Child.main(Child.java:253)
        Hide
        brandon.williams Brandon Williams added a comment -

        So is the conclusion that this patch by itself works fine, but there is a problem with CASSANDRA-2777?

        Show
        brandon.williams Brandon Williams added a comment - So is the conclusion that this patch by itself works fine, but there is a problem with CASSANDRA-2777 ?
        Hide
        silvere Silvère Lestang added a comment -

        No, from my test I arrived to the inverse conclusion: CASSANDRA-2777 seems to works fine (Pig has the good type for my column family) but the bug is still here despite the 2 patches.

        Show
        silvere Silvère Lestang added a comment - No, from my test I arrived to the inverse conclusion: CASSANDRA-2777 seems to works fine (Pig has the good type for my column family) but the bug is still here despite the 2 patches.
        Hide
        brandon.williams Brandon Williams added a comment -

        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.

        Show
        brandon.williams Brandon Williams added a comment - 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.
        Hide
        brandon.williams Brandon Williams added a comment -

        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.

        Show
        brandon.williams Brandon Williams added a comment - 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.
        Hide
        steeve Steeve Morin added a comment -

        Fixed it for me on Pig 0.9 and Cassandra 0.8.6 (Brisk).

        Show
        steeve Steeve Morin added a comment - Fixed it for me on Pig 0.9 and Cassandra 0.8.6 (Brisk).
        Hide
        jeromatron Jeremy Hanna added a comment -

        +1 - if we find any issues with it in production, we'll submit bug reports.

        Show
        jeromatron Jeremy Hanna added a comment - +1 - if we find any issues with it in production, we'll submit bug reports.
        Hide
        brandon.williams Brandon Williams added a comment -

        Committed.

        Show
        brandon.williams Brandon Williams added a comment - Committed.
        Hide
        hudson Hudson added a comment -

        Integrated in Cassandra-0.8 #348 (See https://builds.apache.org/job/Cassandra-0.8/348/)
        Fix handling of integer types in pig.
        Patch by brandonwilliams, reviewed by Jeremy Hanna for CASSANDRA-2810

        brandonwilliams : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1177084
        Files :

        • /cassandra/branches/cassandra-0.8/contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
        Show
        hudson Hudson added a comment - Integrated in Cassandra-0.8 #348 (See https://builds.apache.org/job/Cassandra-0.8/348/ ) Fix handling of integer types in pig. Patch by brandonwilliams, reviewed by Jeremy Hanna for CASSANDRA-2810 brandonwilliams : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1177084 Files : /cassandra/branches/cassandra-0.8/contrib/pig/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java

          People

          • Assignee:
            brandon.williams Brandon Williams
            Reporter:
            silvere Silvère Lestang
            Reviewer:
            Jeremy Hanna
          • Votes:
            1 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development