crunch-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Micah Whitacre (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CRUNCH-601) Short PCollections in SparkPipeline get length null.
Date Wed, 17 Aug 2016 13:38:20 GMT

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

Micah Whitacre updated CRUNCH-601:
----------------------------------
    Attachment: CRUNCH-601.patch

So I thought about it a bit more and came up with an adjustment for the cases where the collection
is non-empty so that when calculating size is it more accurate (e.g. doesn't calculate to
zero).

I don't have a solution for when the collection is actually zero.  The aggregation to get
length[1] is written to produce a count of zero (e.g. the MapFn cleanup will at least emit
a value of 0) but because the PCollection is empty it creates an empty iterator and FirstElementPObject[2]
comes back with null.  

So the tests right now skip the collection size 0 use case but that needs to be solved before
this can be resolved.

[1] - https://github.com/apache/crunch/blob/0b19717d105b58e58c1947eda6b673a387e330d0/crunch-core/src/main/java/org/apache/crunch/lib/Aggregate.java#L95
[2] - https://github.com/apache/crunch/blob/0b19717d105b58e58c1947eda6b673a387e330d0/crunch-core/src/main/java/org/apache/crunch/materialize/pobject/FirstElementPObject.java#L48

> Short PCollections in SparkPipeline get length null.
> ----------------------------------------------------
>
>                 Key: CRUNCH-601
>                 URL: https://issues.apache.org/jira/browse/CRUNCH-601
>             Project: Crunch
>          Issue Type: Bug
>          Components: Spark
>    Affects Versions: 0.13.0
>         Environment: Running in local mode on Mac as well as in a ubuntu 14.04 docker
container
>            Reporter: Mikael Goldmann
>            Priority: Minor
>         Attachments: CRUNCH-601.patch, SmallCollectionLengthTest.java
>
>
> I'll attach a file with a test that I would expect to pass but which fails.
> It creates five PCollection<String> of lengths 0, 1, 2, 3, 4 gets the lengths,
runs the pipeline and prints the lengths. Finally it asserts that all lengths are non-null.
> I would expect it to print lengths 0, 1, 2, 3, 4 and pass.
> What it does is print lengths null, null, null, 3, 4 and fail.
> I think the underlying reason is the use of getSize() on an unmaterialized object and
assuming that when the estimate that getSize() returns is 0, then the PCollection is guaranteed
to be empty, which is false in some cases.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message