[ 
https://issues.apache.org/jira/browse/BEAM-5987?focusedWorklogId=185859&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-185859
 ]

ASF GitHub Bot logged work on BEAM-5987:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 16/Jan/19 16:36
            Start Date: 16/Jan/19 16:36
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on pull request #7091: [BEAM-5987] 
Spark: Share cached side inputs between tasks.
URL: https://github.com/apache/beam/pull/7091#discussion_r248354273
 
 

 ##########
 File path: 
runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java
 ##########
 @@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.util;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollectionView;
+
+/**
+ * Cache deserialized side inputs for executor so every task doesn't need to 
deserialize them again.
+ * Side inputs are stored in {@link Cache} with 5 minutes expireAfterAccess.
+ */
+class SideInputStorage {
+
+  /** JVM deserialized side input cache. */
+  private static final Cache<Key<?>, Optional<?>> materializedSideInputs =
 
 Review comment:
   I am a bit worried on the possible consequence of a collision of the 
`Key<view, window>` tuple in particular if a bad implementation of equals is 
around. This is not relative to this PR but since the state is now static this 
makes the likelihood of this happening bigger.
 
----------------------------------------------------------------
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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 185859)
    Time Spent: 5h 40m  (was: 5.5h)

> 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: 5h 40m
>  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)

Reply via email to