[ https://issues.apache.org/jira/browse/BEAM-5987?focusedWorklogId=188317&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-188317 ]
ASF GitHub Bot logged work on BEAM-5987: ---------------------------------------- Author: ASF GitHub Bot Created on: 22/Jan/19 16:24 Start Date: 22/Jan/19 16:24 Worklog Time Spent: 10m Work Description: mareksimunek commented on pull request #7091: [BEAM-5987] Spark: Share cached side inputs between tasks. URL: https://github.com/apache/beam/pull/7091#discussion_r249856567 ########## File path: runners/spark/src/main/java/org/apache/beam/runners/spark/util/CachedSideInputReader.java ########## @@ -86,9 +55,27 @@ private CachedSideInputReader(SideInputReader delegate) { @Override public <T> T get(PCollectionView<T> view, BoundedWindow window) { @SuppressWarnings("unchecked") - final Map<Key<T>, T> materializedCasted = (Map) materialized; - return materializedCasted.computeIfAbsent( - new Key<>(view, window), key -> delegate.get(view, window)); + final Cache<Key<T>, Optional<T>> materializedCasted = + (Cache) SideInputStorage.getMaterializedSideInputs(); + + Key<T> sideInputKey = new Key<>(view, window); + + try { + Optional<T> optionalResult = + materializedCasted.get( + sideInputKey, + () -> { + final T result = delegate.get(view, window); + LOG.info( + "Caching de-serialized side input for {} of size [{}B] in memory.", + sideInputKey, + SizeEstimator.estimate(result)); + return Optional.ofNullable(result); + }); + return optionalResult.orElse(null); Review comment: Thanks for suggestions, Optional combo would be not very readable. I made my own wrapper where I simply wrap the value so I can put `null` into the cache. https://github.com/apache/beam/pull/7091/files#diff-b123f0f1ca9646966a641a458b74cfbcR92 ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 188317) Time Spent: 7h 50m (was: 7h 40m) > Spark SideInputReader performance > --------------------------------- > > Key: BEAM-5987 > URL: https://issues.apache.org/jira/browse/BEAM-5987 > Project: Beam > Issue Type: Bug > Components: runner-spark > Affects Versions: 2.8.0 > Reporter: David Moravek > Assignee: David Moravek > Priority: Major > Fix For: 2.9.0 > > Attachments: Screen Shot 2018-11-06 at 13.05.36.png > > Time Spent: 7h 50m > Remaining Estimate: 0h > > We did some profiling of a spark job and 90% of the application time was > spent on side input deserialization. > For spark, an easy fix is to cache materialized side inputs per bundle. This > improved running time of the profiled job from 3 hours to 30 minutes. -- This message was sent by Atlassian JIRA (v7.6.3#76005)