You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by "Chris Riccomini (JIRA)" <ji...@apache.org> on 2014/04/21 18:57:14 UTC

[jira] [Updated] (SAMZA-245) Improve SystemConsumers performance

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

Chris Riccomini updated SAMZA-245:
----------------------------------

    Attachment: SAMZA-245.0.patch

Attaching initial patch. RB is up ate:

https://reviews.apache.org/r/20475/

1. Change the API for SystemConsumer to return messages grouped by SSP.
2. Change the API for SystemConsumer to eliminate a max fetch size.
3. Update SystemStreamPartitionIterator to use new API defined in (1) above.
4. Update BlockingEnvelopeMap to use the new API described in (1) above.
5. Switch SystemConsumers to use all java.util data structures.
6. Eliminate all fetch thresholds, max buffer size, etc, since we don't need them due to (2) above.
7. Eliminate neededByChooser and skippingChooser variables, and introduce a map to track empty SystemStreamPartitions grouped by system. The grouping is due to (2) above.
8. Introduce a threshold in SystemConsumers.choose to trigger poll'ing, so we don't poll all system consumers on every choose invocation. Currently the threshold is hard-coded to 1000.
9. Eliminate doubling backoff, since we are already throttling how often we poll using the threshold in (8).
10. Use an iterator instead of Scala for loops to speed things up.
11. Update MessageChooser whenever it picks a message, and also whenever a message arrives for a previously empty SSP.
12. Strip RoundRobinChooser of its safety, because it was taking 14% of CPU in Scala's + set method.

I think that these set of changes maintain the characteristics for the MessageChooser, namely, one SSP update at a time. I ran some preliminary tests, and it didn't seem to receive any duplicates. The trick is that we only add the SSP to the empty map after we've checked it once in the choose method. This is normally the point where we'd add it to the skippingChooser method, and remove it from the neededByChooser method. Instead, we just add it to the empty buffer, and never try to update again until a new envelope arrives for the SSP in question.

Obviously I need to update the tests and write some more to verify that all of our logic still holds.

> Improve SystemConsumers performance
> -----------------------------------
>
>                 Key: SAMZA-245
>                 URL: https://issues.apache.org/jira/browse/SAMZA-245
>             Project: Samza
>          Issue Type: Bug
>          Components: container
>    Affects Versions: 0.6.0
>            Reporter: Chris Riccomini
>             Fix For: 0.8.0
>
>         Attachments: SAMZA-245.0.patch
>
>
> As part of SAMZA-220, a more radical patch was proposed. This patch appears to improve SystemConsumers' performance pretty significantly, while also reducing its complexity. The decision was made to move this change into the 0.8.0 release, rather than the 0.7.0 release, since it's a fairly risky change.
> This ticket is to explore updating SystemConsumers to eliminate almost all loops in order to increase performance in the Samza container.



--
This message was sent by Atlassian JIRA
(v6.2#6252)