ok so let me try again ;-) I don't think that the page size calculation matters apart from hitting the allocation limit earlier if the page size is too large.
If a task is going to need X bytes, it is going to need X bytes. In this case, for at least one of the tasks, X > maxmemory/no_active_tasks at some point during execution. A smaller page size may use the memory more efficiently but would not necessarily avoid this issue. The next question would be: Is the memory limit per task of max_memory/no_active_tasks reasonable? It seems fair but if this limit is reached currently an exception is thrown, maybe the task could wait for no_active_tasks to decrease? I think what causes my test issue is that the 32 tasks don't execute as quickly on my 8 core box so more are active at any one time. I will experiment with the page size calculation to see what effect it has. Cheers, On 16 September 2015 at 06:53, Reynold Xin <r...@databricks.com> wrote: > It is exactly the issue here, isn't it? > > We are using memory / N, where N should be the maximum number of active > tasks. In the current master, we use the number of cores to approximate the > number of tasks -- but it turned out to be a bad approximation in tests > because it is set to 32 to increase concurrency. > > > On Tue, Sep 15, 2015 at 10:47 PM, Pete Robbins <robbin...@gmail.com> > wrote: > >> Oops... I meant to say "The page size calculation is NOT the issue here" >> >> On 16 September 2015 at 06:46, Pete Robbins <robbin...@gmail.com> wrote: >> >>> The page size calculation is the issue here as there is plenty of free >>> memory, although there is maybe a fair bit of wasted space in some pages. >>> It is that when we have a lot of tasks each is only allowed to reach 1/n of >>> the available memory and several of the tasks bump in to that limit. With >>> tasks 4 times the number of cores there will be some contention and so they >>> remain active for longer. >>> >>> So I think this is a test case issue configuring the number of executors >>> too high. >>> >>> On 15 September 2015 at 18:54, Reynold Xin <r...@databricks.com> wrote: >>> >>>> Maybe we can change the heuristics in memory calculation to use >>>> SparkContext.defaultParallelism if it is local mode. >>>> >>>> >>>> On Tue, Sep 15, 2015 at 10:28 AM, Pete Robbins <robbin...@gmail.com> >>>> wrote: >>>> >>>>> Yes and at least there is an override by setting >>>>> spark.sql.test.master to local[8] , in fact local[16] worked on my 8 core >>>>> box. >>>>> >>>>> I'm happy to use this as a workaround but the 32 hard-coded will fail >>>>> running build/tests on a clean checkout if you only have 8 cores. >>>>> >>>>> On 15 September 2015 at 17:40, Marcelo Vanzin <van...@cloudera.com> >>>>> wrote: >>>>> >>>>>> That test explicitly sets the number of executor cores to 32. >>>>>> >>>>>> object TestHive >>>>>> extends TestHiveContext( >>>>>> new SparkContext( >>>>>> System.getProperty("spark.sql.test.master", "local[32]"), >>>>>> >>>>>> >>>>>> On Mon, Sep 14, 2015 at 11:22 PM, Reynold Xin <r...@databricks.com> >>>>>> wrote: >>>>>> > Yea I think this is where the heuristics is failing -- it uses 8 >>>>>> cores to >>>>>> > approximate the number of active tasks, but the tests somehow is >>>>>> using 32 >>>>>> > (maybe because it explicitly sets it to that, or you set it >>>>>> yourself? I'm >>>>>> > not sure which one) >>>>>> > >>>>>> > On Mon, Sep 14, 2015 at 11:06 PM, Pete Robbins <robbin...@gmail.com> >>>>>> wrote: >>>>>> >> >>>>>> >> Reynold, thanks for replying. >>>>>> >> >>>>>> >> getPageSize parameters: maxMemory=515396075, numCores=0 >>>>>> >> Calculated values: cores=8, default=4194304 >>>>>> >> >>>>>> >> So am I getting a large page size as I only have 8 cores? >>>>>> >> >>>>>> >> On 15 September 2015 at 00:40, Reynold Xin <r...@databricks.com> >>>>>> wrote: >>>>>> >>> >>>>>> >>> Pete - can you do me a favor? >>>>>> >>> >>>>>> >>> >>>>>> >>> >>>>>> https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala#L174 >>>>>> >>> >>>>>> >>> Print the parameters that are passed into the getPageSize >>>>>> function, and >>>>>> >>> check their values. >>>>>> >>> >>>>>> >>> On Mon, Sep 14, 2015 at 4:32 PM, Reynold Xin <r...@databricks.com> >>>>>> wrote: >>>>>> >>>> >>>>>> >>>> Is this on latest master / branch-1.5? >>>>>> >>>> >>>>>> >>>> out of the box we reserve only 16% (0.2 * 0.8) of the memory for >>>>>> >>>> execution (e.g. aggregate, join) / shuffle sorting. With a 3GB >>>>>> heap, that's >>>>>> >>>> 480MB. So each task gets 480MB / 32 = 15MB, and each operator >>>>>> reserves at >>>>>> >>>> least one page for execution. If your page size is 4MB, it only >>>>>> takes 3 >>>>>> >>>> operators to use up its memory. >>>>>> >>>> >>>>>> >>>> The thing is page size is dynamically determined -- and in your >>>>>> case it >>>>>> >>>> should be smaller than 4MB. >>>>>> >>>> >>>>>> https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala#L174 >>>>>> >>>> >>>>>> >>>> Maybe there is a place that in the maven tests that we >>>>>> explicitly set >>>>>> >>>> the page size (spark.buffer.pageSize) to 4MB? If yes, we need to >>>>>> find it and >>>>>> >>>> just remove it. >>>>>> >>>> >>>>>> >>>> >>>>>> >>>> On Mon, Sep 14, 2015 at 4:16 AM, Pete Robbins < >>>>>> robbin...@gmail.com> >>>>>> >>>> wrote: >>>>>> >>>>> >>>>>> >>>>> I keep hitting errors running the tests on 1.5 such as >>>>>> >>>>> >>>>>> >>>>> >>>>>> >>>>> - join31 *** FAILED *** >>>>>> >>>>> Failed to execute query using catalyst: >>>>>> >>>>> Error: Job aborted due to stage failure: Task 9 in stage >>>>>> 3653.0 >>>>>> >>>>> failed 1 times, most recent failure: Lost task 9.0 in stage >>>>>> 3653.0 (TID >>>>>> >>>>> 123363, localhost): java.io.IOException: Unable to acquire >>>>>> 4194304 bytes of >>>>>> >>>>> memory >>>>>> >>>>> at >>>>>> >>>>> >>>>>> org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPage(UnsafeExternalSorter.java:368) >>>>>> >>>>> >>>>>> >>>>> >>>>>> >>>>> This is using the command >>>>>> >>>>> build/mvn -Pyarn -Phadoop-2.2 -Phive -Phive-thriftserver test >>>>>> >>>>> >>>>>> >>>>> >>>>>> >>>>> I don't see these errors in any of the amplab jenkins builds. >>>>>> Do those >>>>>> >>>>> builds have any configuration/environment that I may be >>>>>> missing? My build is >>>>>> >>>>> running with whatever defaults are in the top level pom.xml, eg >>>>>> -Xmx3G. >>>>>> >>>>> >>>>>> >>>>> I can make these tests pass by setting >>>>>> spark.shuffle.memoryFraction=0.6 >>>>>> >>>>> in the HiveCompatibilitySuite rather than the default 0.2 value. >>>>>> >>>>> >>>>>> >>>>> Trying to analyze what is going on with the test it is related >>>>>> to the >>>>>> >>>>> number of active tasks, which seems to rise to 32, and so the >>>>>> >>>>> ShuffleMemoryManager allows less memory per task even though >>>>>> most of those >>>>>> >>>>> tasks do not have any memory allocated to them. >>>>>> >>>>> >>>>>> >>>>> Has anyone seen issues like this before? >>>>>> >>>> >>>>>> >>>> >>>>>> >>> >>>>>> >> >>>>>> > >>>>>> >>>>>> >>>>>> >>>>>> -- >>>>>> Marcelo >>>>>> >>>>> >>>>> >>>> >>> >> >