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] [Commented] (CRUNCH-601) Short PCollections in SparkPipeline get length null.
Date Wed, 17 Aug 2016 02:30:20 GMT

    [ https://issues.apache.org/jira/browse/CRUNCH-601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15423764#comment-15423764
] 

Micah Whitacre commented on CRUNCH-601:
---------------------------------------

So I think I figured out the reason but not really sure of the fix just yet.  When doing a
.length()[1] call it then calls Aggregate.length(this) and does this[2].

The issue is that the "count" PCollection in that method it has 4 different parents.  It does
calculate the expected size of the PCollection to proactively materialize or not by using
the scaleFactor * parent.  So for a collection of size 1 it is essentially calculating (.99f
* (long)(.99f * (long)(.99f * (long)(.99f *1)))).  And casting to long causes it to round
down.  so the first .99f * 1 = 0 when cast to a long.  So the smaller values are invalid because
the scale factor makes their size go to zero which then makes the materialize call return
the empty PCollection.

[1] - https://github.com/apache/crunch/blob/0b19717d105b58e58c1947eda6b673a387e330d0/crunch-core/src/main/java/org/apache/crunch/impl/dist/collect/PCollectionImpl.java#L284
[2] - https://github.com/apache/crunch/blob/0b19717d105b58e58c1947eda6b673a387e330d0/crunch-core/src/main/java/org/apache/crunch/lib/Aggregate.java#L94

> 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: 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