You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Сергей Романов <ro...@inbox.ru.INVALID> on 2016/09/07 09:33:17 UTC

Re[10]: Spark 2.0: SQL runs 5x times slower when adding 29th field to aggregation.

Thank you, Yong, it looks great.

I had added following lines to spark-defaults.conf and now my original SQL query runs much faster.
spark.executor.extraJavaOptions -XX:-DontCompileHugeMethods
spark.driver.extraJavaOptions -XX:-DontCompileHugeMethods
Can you recommend these configuration settings for production mode? Will it have any side-effects? Will it supersede  SPARK-17115?
SQL:
SELECT `publisher_id` AS `publisher_id`, SUM(`conversions`) AS `conversions`, SUM(`dmp_rapleaf_margin`) AS `dmp_rapleaf_margin`, SUM(`pvc`) AS `pvc`, SUM(`dmp_nielsen_payout`) AS `dmp_nielsen_payout`, SUM(`fraud_clicks`) AS `fraud_clicks`, SUM(`impressions`) AS `impressions`, SUM(`conv_prob`) AS `conv_prob`, SUM(`dmp_liveramp_payout`) AS `dmp_liveramp_payout`, SUM(`decisions`) AS `decisions`, SUM(`fraud_impressions`) AS `fraud_impressions`, SUM(`advertiser_spent`) AS `advertiser_spent`, SUM(`actual_ssp_fee`) AS `actual_ssp_fee`, SUM(`dmp_nielsen_margin`) AS `dmp_nielsen_margin`, SUM(`first_impressions`) AS `first_impressions`, SUM(`clicks`) AS `clicks`, SUM(`second_price`) AS `second_price`, SUM(`click_prob`) AS `click_prob`, SUM(`clicks_static`) AS `clicks_static`, SUM(`expected_payout`) AS `expected_payout`, SUM(`bid_price`) AS `bid_price`, SUM(`noads`) AS `noads`, SUM(`e`) AS `e`, SUM(`e`) as `e2`, SUM(`publisher_revenue`) AS `publisher_revenue`, SUM(`dmp_liveramp_margin`) AS `dmp_liveramp_margin`, SUM(`actual_pgm_fee`) AS `actual_pgm_fee`, SUM(`dmp_rapleaf_payout`) AS `dmp_rapleaf_payout`, SUM(`dd_convs`) AS `dd_convs`, SUM(`actual_dsp_fee`) AS `actual_dsp_fee` FROM `slicer`.`573_slicer_rnd_13` WHERE dt = '2016-07-28' GROUP BY `publisher_id` LIMIT 30;
Original:
30 rows selected (10.047 seconds)
30 rows selected (10.612 seconds)
30 rows selected (9.935 seconds)
With -XX:-DontCompileHugeMethods:
30 rows selected (1.086 seconds)
30 rows selected (1.051 seconds)
30 rows selected (1.073 seconds)

>Среда,  7 сентября 2016, 0:35 +03:00 от Yong Zhang <ja...@hotmail.com>:
>
>This is an interesting point.
>
>I tested with originally data with Spark 2.0 release, I can get the same statistic output in the originally email like following:
>
>50 1.77695393562
>51 0.695149898529
>52 0.638142108917
>53 0.647341966629
>54 0.663456916809
>55 0.629166126251
>56 0.644149065018
>57 0.661190986633
>58 2.6616499424
>59 2.6137509346
>60 2.71165704727
>61 2.63473916054
>
>Then I tested with your suggestion:
>
>spark/bin/pyspark --driver-java-options '-XX:-DontCompileHugeMethods'
>
>Run the same test code, and here is the output:
>
>50 1.77180695534
>51 0.679394006729
>52 0.629493951797
>53 0.62108206749
>54 0.637018918991
>55 0.640591144562
>56 0.649922132492
>57 0.652480125427
>58 0.636356830597
>59 0.667215824127
>60 0.643863916397
>61 0.669810056686
>62 0.664624929428
>63 0.682888031006
>64 0.691393136978
>65 0.690823078156
>66 0.70525097847
>67 0.724694013596
>68 0.737638950348
>69 0.749594926834
>
>
>Yong
>
>----------------------------------------------------------------------
>From: Davies Liu < davies@databricks.com >
>Sent: Tuesday, September 6, 2016 2:27 PM
>To: Сергей Романов
>Cc: Gavin Yue; Mich Talebzadeh; user
>Subject: Re: Re[8]: Spark 2.0: SQL runs 5x times slower when adding 29th field to aggregation.
> 
>I think the slowness is caused by generated aggregate method has more
>than 8K bytecodes, than it's not JIT compiled, became much slower.
>
>Could you try to disable the DontCompileHugeMethods by:
>
>-XX:-DontCompileHugeMethods
>
>On Mon, Sep 5, 2016 at 4:21 AM, Сергей Романов
>< romanovsa@inbox.ru.invalid > wrote:
>> Hi, Gavin,
>>
>> Shuffling is exactly the same in both requests and is minimal. Both requests
>> produces one shuffle task. Running time is the only difference I can see in
>> metrics:
>>
>> timeit.timeit(spark.read.csv('file:///data/dump/test_csv',
>> schema=schema).groupBy().sum(*(['dd_convs'] * 57) ).collect, number=1)
>> 0.713730096817
>>  {
>>     "id" : 368,
>>     "name" : "duration total (min, med, max)",
>>     "value" : "524"
>>   }, {
>>     "id" : 375,
>>     "name" : "internal.metrics.executorRunTime",
>>     "value" : "527"
>>   }, {
>>     "id" : 391,
>>     "name" : "internal.metrics.shuffle.write.writeTime",
>>     "value" : "244495"
>>   }
>>
>> timeit.timeit(spark.read.csv('file:///data/dump/test_csv',
>> schema=schema).groupBy().sum(*(['dd_convs'] * 58) ).collect, number=1)
>> 2.97951102257
>>
>>   }, {
>>     "id" : 469,
>>     "name" : "duration total (min, med, max)",
>>     "value" : "2654"
>>   }, {
>>     "id" : 476,
>>     "name" : "internal.metrics.executorRunTime",
>>     "value" : "2661"
>>   }, {
>>     "id" : 492,
>>     "name" : "internal.metrics.shuffle.write.writeTime",
>>     "value" : "371883"
>>   }, {
>>
>> Full metrics in attachment.
>>
>> Суббота, 3 сентября 2016, 19:53 +03:00 от Gavin Yue
>> < yue.yuanyuan@gmail.com >:
>>
>>
>> Any shuffling?
>>
>>
>> On Sep 3, 2016, at 5:50 AM, Сергей Романов < romanovsa@inbox.ru.INVALID >
>> wrote:
>>
>> Same problem happens with CSV data file, so it's not parquet-related either.
>>
>> Welcome to
>>       ____              __
>>      / __/__  ___ _____/ /__
>>     _\ \/ _ \/ _ `/ __/  '_/
>>    /__ / .__/\_,_/_/ /_/\_\   version 2.0.0
>>       /_/
>>
>> Using Python version 2.7.6 (default, Jun 22 2015 17:58:13)
>> SparkSession available as 'spark'.
>>>>> import timeit
>>>>> from pyspark.sql.types import *
>>>>> schema = StructType([StructField('dd_convs', FloatType(), True)])
>>>>> for x in range(50, 70): print x,
>>>>> timeit.timeit(spark.read.csv('file:///data/dump/test_csv',
>>>>> schema=schema).groupBy().sum(*(['dd_convs'] * x) ).collect, number=1)
>> 50 0.372850894928
>> 51 0.376906871796
>> 52 0.381325960159
>> 53 0.385444164276
>> 54 0.386877775192
>> 55 0.388918161392
>> 56 0.397624969482
>> 57 0.391713142395
>> 58 2.62714004517
>> 59 2.68421196938
>> 60 2.74627685547
>> 61 2.81081581116
>> 62 3.43532109261
>> 63 3.07742786407
>> 64 3.03904604912
>> 65 3.01616096497
>> 66 3.06293702126
>> 67 3.09386610985
>> 68 3.27610206604
>> 69 3.2041969299
>>
>> Суббота, 3 сентября 2016, 15:40 +03:00 от Сергей Романов
>> < romanovsa@inbox.ru.INVALID >:
>>
>> Hi,
>>
>> I had narrowed down my problem to a very simple case. I'm sending 27kb
>> parquet in attachment. ( file:///data/dump/test2 in example)
>>
>> Please, can you take a look at it? Why there is performance drop after 57
>> sum columns?
>>
>> Welcome to
>>       ____              __
>>      / __/__  ___ _____/ /__
>>     _\ \/ _ \/ _ `/ __/  '_/
>>    /__ / .__/\_,_/_/ /_/\_\   version 2.0.0
>>       /_/
>>
>> Using Python version 2.7.6 (default, Jun 22 2015 17:58:13)
>> SparkSession available as 'spark'.
>>>>> import timeit
>>>>> for x in range(70): print x,
>>>>> timeit.timeit(spark.read.parquet('file:///data/dump/test2').groupBy().sum(*(['dd_convs']
>>>>> * x) ).collect, number=1)
>> ...
>> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>> SLF4J: Defaulting to no-operation (NOP) logger implementation
>> SLF4J: See  http://www.slf4j.org/codes.html#StaticLoggerBinder for further
>> details.
>> 0 1.05591607094
>> 1 0.200426101685
>> 2 0.203800916672
>> 3 0.176458120346
>> 4 0.184863805771
>> 5 0.232321023941
>> 6 0.216032981873
>> 7 0.201778173447
>> 8 0.292424917221
>> 9 0.228524923325
>> 10 0.190534114838
>> 11 0.197028160095
>> 12 0.270443916321
>> 13 0.429781913757
>> 14 0.270851135254
>> 15 0.776989936829
>> 16 0.233337879181
>> 17 0.227638959885
>> 18 0.212944030762
>> 19 0.2144780159
>> 20 0.22200012207
>> 21 0.262261152267
>> 22 0.254227876663
>> 23 0.275084018707
>> 24 0.292124032974
>> 25 0.280488014221
>> 16/09/03 15:31:28 WARN Utils: Truncated the string representation of a plan
>> since it was too large. This behavior can be adjusted by setting
>> 'spark.debug.maxToStringFields' in SparkEnv.conf.
>> 26 0.290093898773
>> 27 0.238478899002
>> 28 0.246420860291
>> 29 0.241401195526
>> 30 0.255286931992
>> 31 0.42702794075
>> 32 0.327946186066
>> 33 0.434395074844
>> 34 0.314198970795
>> 35 0.34576010704
>> 36 0.278323888779
>> 37 0.289474964142
>> 38 0.290827989578
>> 39 0.376291036606
>> 40 0.347742080688
>> 41 0.363158941269
>> 42 0.318687915802
>> 43 0.376327991486
>> 44 0.374994039536
>> 45 0.362971067429
>> 46 0.425967931747
>> 47 0.370860099792
>> 48 0.443903923035
>> 49 0.374128103256
>> 50 0.378985881805
>> 51 0.476850986481
>> 52 0.451028823853
>> 53 0.432540893555
>> 54 0.514838933945
>> 55 0.53990483284
>> 56 0.449142932892
>> 57 0.465240001678 // 5x slower after 57 columns
>> 58 2.40412116051
>> 59 2.41632795334
>> 60 2.41812801361
>> 61 2.55726218224
>> 62 2.55484509468
>> 63 2.56128406525
>> 64 2.54642391205
>> 65 2.56381797791
>> 66 2.56871509552
>> 67 2.66187620163
>> 68 2.63496208191
>> 69 2.81545996666
>>
>>
>> Sergei Romanov
>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe e-mail:  user-unsubscribe@spark.apache.org
>>
>> Sergei Romanov
>>
>> <bad.csv.tgz>
>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe e-mail:  user-unsubscribe@spark.apache.org
>>
>>
>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe e-mail:  user-unsubscribe@spark.apache.org
>
>---------------------------------------------------------------------
>To unsubscribe e-mail:  user-unsubscribe@spark.apache.org
>