You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@pig.apache.org by Jeremy Hanna <je...@gmail.com> on 2011/04/15 23:44:06 UTC

COUNT sometimes returning a float value?

I have been getting strange errors in my pig script and narrowed it down a bit and found that when I do a COUNT, sometimes it returns a float, but most of the time it returns a long.  Some example output of the result column that came from a COUNT is below.  Any reason why this would happen?

The error that started me down this path was in my next step, I cast the count value as a chararray and it gives the error "java.lang.ClassCastException: java.lang.Float cannot be cast to java.lang.Long" which made no sense to me since we weren't casting anything in our script to a long.

(0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)

Re: COUNT sometimes returning a float value?

Posted by Jeremy Hanna <je...@gmail.com>.
I looked at previous outputs and saw that both bags of attributes 5 and 6 are currently empty for all records - which is expected because we're working with data that doesn't yet have those.  So it might be choking on handling that case.  The COUNT values that return 0 and 0.0 are for attr6_bag - "COUNT(records_with_attr6s.record_key) AS attr6_count,"

Here is the part of the script that produces the count:

records_clean = FOREACH records GENERATE 
    record_key, type, snapshot_id,
    (attr1_id is null ? -1 : attr1_id) AS attr1_id,
    (attr2_id is null ? -1 : attr2_id) AS attr2_id,
    (attr3_id is null ? -1 : attr3_id) AS attr3_id,
    (attr4_id is null ? -1 : attr4_id) AS attr4_id,
    attr5_bag,
    attr6_bag;

records_with_attr5s = FILTER records_clean BY NOT(IsEmpty(attr5_bag));
records_with_attr6s = FILTER records_clean BY NOT(IsEmpty(attr6_bag));

records_by_account = COGROUP records_clean BY
    (snapshot_id, type, attr1_id, attr2_id, attr3_id, attr4_id),
    records_with_attr5s BY (snapshot_id, type, attr1_id, attr2_id, attr3_id, attr4_id),
    records_with_attr6s BY (snapshot_id, type, attr1_id, attr2_id, attr3_id, attr4_id) PARALLEL 4;

raw_result = FOREACH records_by_account GENERATE
    FLATTEN(STRSPLIT(group.snapshot_id, ':', 2)) AS (account_temp, bucket_temp),
    group.type, 
    group.attr1_id,
    group.attr2_id,
    group.attr3_id,
    group.attr4_id,
    COUNT(records_clean.record_key) AS total_count,
    COUNT(records_with_attr5s.record_key) AS attr5_count,
    COUNT(records_with_attr6s.record_key) AS attr6_count,
    (float) COUNT(records_with_attr5s.record_key) / (float) COUNT(records_clean.record_key) AS attr5_percent,
    (float) COUNT(records_with_attr6s.record_key) / (float) COUNT(records_clean.record_key) AS attr6_percent;

and the full stack trace:
java.lang.ClassCastException: java.lang.Float cannot be cast to java.lang.Long
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POCast.getNext(POCast.java:699)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.processPlan(POForEach.java:367)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:291)
	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Reduce.runPipeline(PigMapReduce.java:433)
	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Reduce.processOnePackageOutput(PigMapReduce.java:401)
	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Reduce.reduce(PigMapReduce.java:381)
	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce$Reduce.reduce(PigMapReduce.java:251)
	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:176)
	at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:571)
	at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:413)
	at org.apache.hadoop.mapred.Child$4.run(Child.java:240)
	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:1115)
	at org.apache.hadoop.mapred.Child.main(Child.java:234)


On Apr 15, 2011, at 7:13 PM, Thejas M Nair wrote:

> This is strange. Looking at COUNT code, there does not seem to be anyway it could return a float.
> Do you have some example data/query that can be used to reproduce this ?
> Can you paste the entire stacktrace of the ClassCastException ?
> Do you have something like a bincond which might be returning different results for different rows ?
> 
> -Thejas
> 
> 
> 
> 
> On 4/15/11 2:44 PM, "Jeremy Hanna" <je...@gmail.com> wrote:
> 
> I have been getting strange errors in my pig script and narrowed it down a bit and found that when I do a COUNT, sometimes it returns a float, but most of the time it returns a long.  Some example output of the result column that came from a COUNT is below.  Any reason why this would happen?
> 
> The error that started me down this path was in my next step, I cast the count value as a chararray and it gives the error "java.lang.ClassCastException: java.lang.Float cannot be cast to java.lang.Long" which made no sense to me since we weren't casting anything in our script to a long.
> 
> (0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0)
> (0)
> (0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0.0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0.0)
> (0.0)
> (0.0)
> (0.0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0)
> (0) 
> 
> 
> 
> -- 
> 


Re: COUNT sometimes returning a float value?

Posted by Thejas M Nair <te...@yahoo-inc.com>.
This is strange. Looking at COUNT code, there does not seem to be anyway it could return a float.
Do you have some example data/query that can be used to reproduce this ?
Can you paste the entire stacktrace of the ClassCastException ?
Do you have something like a bincond which might be returning different results for different rows ?

-Thejas




On 4/15/11 2:44 PM, "Jeremy Hanna" <je...@gmail.com> wrote:

I have been getting strange errors in my pig script and narrowed it down a bit and found that when I do a COUNT, sometimes it returns a float, but most of the time it returns a long.  Some example output of the result column that came from a COUNT is below.  Any reason why this would happen?

The error that started me down this path was in my next step, I cast the count value as a chararray and it gives the error "java.lang.ClassCastException: java.lang.Float cannot be cast to java.lang.Long" which made no sense to me since we weren't casting anything in our script to a long.

(0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0.0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0.0)
(0.0)
(0.0)
(0.0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)
(0)



--