You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Yong Zhang <ja...@hotmail.com> on 2016/02/23 23:44:49 UTC

Spark 1.5.2, DataFrame broadcast join, OOM

Hi, 
I am testing the Spark 1.5.2 using a 4 nodes cluster, with 64G memory each, and one is master and 3 are workers. I am using Standalone mode.
Here is my spark-env.sh for settings:
export SPARK_LOCAL_DIRS=/data1/spark/local,/data2/spark/local,/data3/spark/localexport SPARK_MASTER_WEBUI_PORT=8081export SPARK_MASTER_IP=hostnameexport SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores=3"export SPARK_WORKER_MEMORY=24gexport SPARK_WORKER_CORES=6export SPARK_WORKER_DIR=/tmp/spark/workexport SPARK_DRIVER_MEMORY=4gexport SPARK_EXECUTOR_MEMORY=2gexport SPARK_DAEMON_MEMORY=1g
I start my spark-shell in the following command: /opt/spark/bin/spark-shell --conf spark-executor_memory=8g --conf total_executor_cores=12
So each executor will have 8G as total heap, and allow up to 12/3=4 concurrence running.
Now I have a join between 2 dataframes:
val loadRaw = sqlContext.read.parquet("parquet_file_path")val trialRaw = loadRaw.filter(instr(loadRaw("event_list"), "202") > 0).select("soid_e1","visid_high","visid_low","date_time")
// trialRaw.count is 1106, with 4 columns, very small data
val historyRaw = sqlContext.read.parquet("multi path parquet files")
// historyRaw.count is 13.5M, but with 500 columns
If I run the join this way:
val join1 = trialRaw.join(historyRaw, trialRaw("visid_high") <=> historyRaw("visid_high") &&  trialRaw("visid_low") <=> historyRaw("visid_low") && trialRaw("date_time") > historyRaw("date_time"))join1.count
I can get the result after 20 minutes, in the above setting on this cluster. But I know the best way is to do the broadcast join, as the trailRaw is so small, so I did this:
val join2 = historyRaw.join(broadcast(trialRaw), trialRaw("visid_high") <=> historyRaw("visid_high") &&  trialRaw("visid_low") <=> historyRaw("visid_low") && trialRaw("date_time") > historyRaw("date_time"))join2.count
To my surprise, in this case the executors got the java.lang.OutOfMemoryError: GC overhead limit exceeded, I know the historyRaw is around 500 columns, which is big. My end result does want to include all the 500 columns in the join, as long as the join conditions match. But the count of row will be limited. Also the trialRaw is very small, only 1106 rows with 4 columns. This is the only data should be shipped to all the executors, and they just read the Parquet data and match by the join conditions.Why in this case it runs out of memory?
1) I know the OOM is not in Driver, as it has 4G heap, and the OOM error comes from the executor stderr2) I know I can add more heap, but I really want to know is that under the same settings, why join1 works, but join2 failed with such small data broadcasting?3) Is this related to how the parquet files being read by Spark?4) It is my first time trying this broadcast join in Spark dataframe, as it is only available since 1.5. So maybe there is some pitfall I don't understand how it works.
This is the stdout, it shows the full GC, then OOM:
2016-02-22T11:54:44.014-0500: [Full GC [PSYoungGen: 1994715K->1943403K(2395136K)] [ParOldGen: 5592413K->5592495K(5592576K)] 7587129K->7535898K(7987712K) [PSPermGen: 63554K->63548K(64000K)], 3.7543230 secs] [Times: user=50.72 sys=0.06, real=3.75 secs] 
2016-02-22T11:54:47.803-0500: [Full GC [PSYoungGen: 1994752K->1960598K(2395136K)] [ParOldGen: 5592495K->5592485K(5592576K)] 7587247K->7553083K(7987712K) [PSPermGen: 63690K->63587K(64000K)], 30.9218790 secs] [Times: user=543.07 sys=2.50, real=30.91 secs] 
Heap
 PSYoungGen      total 2395136K, used 1988191K [0x0000000755500000, 0x0000000800000000, 0x0000000800000000)
  eden space 1994752K, 99% used [0x0000000755500000,0x00000007cea97c30,0x00000007cf100000)
  from space 400384K, 0% used [0x00000007cf100000,0x00000007cf100000,0x00000007e7800000)
  to   space 390656K, 0% used [0x00000007e8280000,0x00000007e8280000,0x0000000800000000)
 ParOldGen       total 5592576K, used 5592485K [0x00000005fff80000, 0x0000000755500000, 0x0000000755500000)
  object space 5592576K, 99% used [0x00000005fff80000,0x00000007554e94e8,0x0000000755500000)
 PSPermGen       total 64000K, used 63710K [0x00000005eff80000, 0x00000005f3e00000, 0x00000005fff80000)
  object space 64000K, 99% used [0x00000005eff80000,0x00000005f3db7ac8,0x00000005f3e00000)
This is the executor log:16/02/22 11:53:41 INFO hadoop.InternalParquetRecordReader: RecordReader initialized will read a total of 0 records.
16/02/22 11:54:27 INFO parquet.ParquetRelation$$anonfun$buildScan$1$$anon$1: Input split: ParquetInputSplit{part: hdfs://host:9000/data/event_parquet/2015/08/21/event_parquet-2015-08-21-20150831195822.parquet start: 134217728 end: 268435456 length: 134217728 hosts: []}
16/02/22 11:54:42 WARN hadoop.ParquetRecordReader: Can not initialize counter due to context is not a instance of TaskInputOutputContext, but is org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
16/02/22 11:54:42 ERROR executor.Executor: Managed memory leak detected; size = 33554432 bytes, TID = 206
16/02/22 11:54:42 ERROR executor.Executor: Exception in task 1.0 in stage 4.0 (TID 206)
java.lang.OutOfMemoryError: GC overhead limit exceeded
	at sun.nio.cs.StreamEncoder.write(StreamEncoder.java:133)
	at java.io.OutputStreamWriter.write(OutputStreamWriter.java:220)
	at java.io.Writer.write(Writer.java:157)
Below is the join plan:
scala> join2.explain16/02/23 17:11:03 INFO storage.MemoryStore: ensureFreeSpace(156640) called with curMem=252597, maxMem=222302306316/02/23 17:11:03 INFO storage.MemoryStore: Block broadcast_5 stored as values in memory (estimated size 153.0 KB, free 2.1 GB)16/02/23 17:11:03 INFO storage.MemoryStore: ensureFreeSpace(16125) called with curMem=409237, maxMem=222302306316/02/23 17:11:03 INFO storage.MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 15.7 KB, free 2.1 GB)16/02/23 17:11:03 INFO storage.BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.20.180.71:46082 (size: 15.7 KB, free: 2.1 GB)16/02/23 17:11:03 INFO spark.SparkContext: Created broadcast 5 from explain at <console>:2816/02/23 17:11:03 INFO storage.MemoryStore: ensureFreeSpace(156640) called with curMem=425362, maxMem=222302306316/02/23 17:11:03 INFO storage.MemoryStore: Block broadcast_6 stored as values in memory (estimated size 153.0 KB, free 2.1 GB)16/02/23 17:11:03 INFO storage.MemoryStore: ensureFreeSpace(16125) called with curMem=582002, maxMem=222302306316/02/23 17:11:03 INFO storage.MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 15.7 KB, free 2.1 GB)16/02/23 17:11:03 INFO storage.BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.20.180.71:46082 (size: 15.7 KB, free: 2.1 GB)16/02/23 17:11:03 INFO spark.SparkContext: Created broadcast 6 from explain at <console>:28== Physical Plan ==Filter (((visid_high#460L <=> visid_high#948L) && (visid_low#461L <=> visid_low#949L)) && (date_time#25L > date_time#513L)) CartesianProduct  Scan ParquetRelation[hdfs://xxx/data/event_parquet/2015/08/13,hdfs://xxx/data/event_parquet/2015/08/14,hdfs://xxx/data/event_parquet/2015/08/21][accept_language#488,browser#489L,browser_height#490,browser_width#491,campaign#492,c_color#493,channel#494,click_action#495,click_action_type#496,click_context#497,click_context_type#498,click_sourceId#499L,click_tag#500,code_ver#501,color#502,connection_type#503,cookies#504,country#505,ct_connect_type#506,currency#507,curr_factor#508,curr_rate#509,cust_hit_time_gmt#510L,cust_visid#511,daily_visitor#512,date_time#513L,domain#514,duplicated_from#515,duplicate_events#516,duplicate_purchase#517,soid_e1#518,lpuid_e2#519,customer_status_e3#520,partner_uid_e4#521,ab_split_group_e5#522,application_e6#523,page_name_e7#524,originating_page_e8#525,email_signup_e9#526,survey_signup_e10#527,event_signup_e11#528,email_paying_e12#529,survey_paying_e13#530,event_paying_e14#531,image_hosting_paying_e15#532,archive_paying_e16#533,internal_search_query_e17#534,internal_search_page_e18#535,download_file_name_e19#536,download_file_page_e20#537,campaign_stacking_e21#538,improveit_category_e22#539,improveit_keyword_e23#540,omtr_vi_id_e24#541,aprimo_id_e25#542,link_name_e26#543,user_agent_e27#544,page_type_desc_e28#545,site_region_language_e29#546,social_campaigns_signup_e30#547,session_history_e31#548,external_data_source_type_e32#549,rmc_code_e33#550,video_name_e34#551,video_segment_e35#552,video_content_e36#553,free_user_prod_name_e37#554,web_voc_id_e38#555,ic_code_e39#556,hit_count_e40#557,signup_form_orig_page_e41#558,e42#559,e43#560,e44#561,pyament_plan_type_e45#562,purchased_product_name_e46#563,purchased_product_category_e47#564,link_loc_name_e48#565,link_pg_loc_name_e49#566,email_test_counter_e50#567,email_preview_counter_e51#568,email_send_counter_e52#569,time_emcf_tpicker_e53#570,time_emcf_editor_e54#571,time_emcf_scheduler_e55#572,time_emcf_build_e56#573,time_emcf_send_success_e57#574,timestamp_e58#575,templates_returned_e59#576,email_agent_id_e60#577,template_name_e61#578,template_hit_counter_e62#579,template_taxonomy_e63#580,email_home_taxonomy_e64#581,email_home_hit_counter_e65#582,editor_taxonomy_t66#583,editor_block_detail_e67#584,editor_block_title_e68#585,editor_hit_counter_e69#586,email_insert_detail_e70#587,email_creation_source_e71#588,e72#589,e73#590,e74#591,adlens_ef_id_e75#592,event_list#593,exclude_hit#594,first_hit_pagename#595,first_hit_page_url#596,first_hit_referrer#597,first_hit_time_gmt#598L,geo_city#599,geo_country#600,geo_dma#601,geo_region#602,geo_zip#603,hier1#604,hier2#605,hier3#606,hier4#607,hier5#608,hitid_high#609L,hitid_low#610L,hit_source#611,hit_time_gmt#612L,homepage#613,hourly_visitor#614,ip#615,ip2#616,java_enabled#617,javascript#618,j_jscript#619,language#620,last_hit_time_gmt#621L,last_purchase_num#622,last_purchase_time_gmt#623L,mobile_id#624L,monthly_visitor#625L,mvvar1#626,mvvar2#627,mvvar3#628,namespace#629,new_visit#630,os#631L,page_event#632,page_event_var1#633,page_event_var2#634,page_event_var3#635,page_name#636,page_type#637,page_url#638,paid_search#639,partner_plugins#640,persistent_cookie#641,plugins#642,post_browser_height#643,post_browser_width#644,post_campaign#645,post_channel#646,post_cookies#647,post_currency#648,post_cust_hit_time_gmt#649L,post_cust_visid#650,soid_pe1#651,lpuid_pe2#652,customer_status_pe3#653,partner_uid_pe4#654,ab_split_group_pe5#655,application_pe6#656,page_name_pe7#657,pe8#658,pe9#659,pe10#660,pe11#661,pe12#662,pe13#663,pe14#664,pe15#665,pe16#666,pe17#667,pe18#668,pe19#669,pe20#670,campaign_stacking_pe21#671,category_name_pe22#672,improvedit_keyword_pe23#673,omtr_vi_id_pe24#674,aprimo_id_pe25#675,link_name_pe26#676,pe27#677,pe28#678,site_region_language_pe29#679,pe30#680,session_history_pe31#681,pe32#682,rmc_code_pe33#683,video_name_pe34#684,pe35#685,pe36#686,pe37#687,pe38#688,ic_code_pe39#689,template_hit_crt_pe40#690,pe41#691,pe42#692,pe43#693,pe44#694,payment_plan_type_pe45#695,purchased_product_name_pe46#696,purchased_product_category_pe47#697,link_loc_name_pe48#698,link_pg_loc_name_pe49#699,email_test_ctr_pe50#700,email_preview_ctr_pe51#701,email_send_ctr_pe52#702,time_emcf_tpicker_pe53#703,time_emcf_editor_pe54#704,time_emcf_scheduler_pe55#705,time_emcf_build_pe56#706,time_emcf_send_success_pe57#707,pe58#708,pe59#709,email_agent_id_pe60#710,template_name_pe61#711,template_hit_ctr_pe62#712,template_taxonomy_pe63#713,email_home_taxonomy_pe64#714,email_home_hit_crt_pe65#715,editor_taxonomy_pe66#716,editor_block_detail_pe67#717,editor_block_title_pe68#718,editor_hit_crt_pe69#719,editor_insert_detail_pe70#720,email_creation_source_pe71#721,pe72#722,pe73#723,pe74#724,adlens_ef_id_pe75#725,post_event_list#726,post_hier1#727,post_hier2#728,post_hier3#729,post_hier4#730,post_hier5#731,post_java_enabled#732,post_keywords#733,post_mvvar1#734,post_mvvar2#735,post_mvvar3#736,post_page_event#737,post_page_event_var1#738,post_page_event_var2#739,post_page_event_var3#740,post_pagename#741,post_pagename_no_url#742,post_page_type#743,post_page_url#744,post_partner_plugins#745,post_persistent_cookie#746,post_product_list#747,soid_pp1#748,lpuid_pp2#749,custom_status_pp3#750,internal_external_pp4#751,ab_split_group_pp5#752,application_pp6#753,internal_search_query_pp7#754,pp8#755,pp9#756,pp10#757,pp11#758,sops_pp12#759,pp13#760,pp14#761,pp15#762,pp16#763,pp17#764,pp18#765,signup_form_pp19#766,pp20#767,pp21#768,category_name_pp22#769,improveit_keyword_pp23#770,pp24#771,sops_hero_name_pp25#772,video_name_pp26#773,pp27#774,topnav_link_name_pp28#775,pp29#776,pp30#777,pp31#778,pp32#779,pp33#780,pp34#781,pp35#782,pp36#783,pp37#784,pp38#785,pp39#786,link_name_pp40#787,link_loc_name_pp41#788,link_pg_loc_name_pp42#789,pp43#790,pp44#791,pp45#792,pp46#793,pp47#794,pp48#795,template_taxonomy_pp49#796,template_name_pp50#797,email_home_taxonomy_pp51#798,editor_taxonomy_pp52#799,kb_details_pp53#800,pp54#801,pp55#802,pp56#803,pp57#804,pp58#805,pp59#806,pp60#807,pp61#808,pp62#809,pp63#810,pp64#811,pp65#812,pp66#813,pp67#814,pp68#815,pp69#816,pp70#817,pp71#818,pp72#819,pp73#820,pp74#821,pp75#822,post_purchaseid#823,post_referrer#824,post_search_engine#825,post_state#826,post_survey#827,post_tnt#828,post_transactionid#829,post_t_time_info#830,post_visid_high#831L,post_visid_low#832L,post_visid_type#833,post_zip#834,p_plugins#835,prev_page#836L,product_list#837,product_merchandising#838,soid_p1#839,p2#840,p3#841,p4#842,p5#843,p6#844,p7#845,p8#846,p9#847,p10#848,p11#849,p12#850,p13#851,p14#852,p15#853,p16#854,p17#855,p18#856,signup_form_p19#857,p20#858,p21#859,improveit_category_p22#860,improveit_keyword_p23#861,test_drive_type_p24#862,sops_hero_name_p25#863,video_name_p26#864,previous_page_name_p27#865,topnav_link_name_p28#866,p29#867,marketplace_detail_p30#868,p31#869,p32#870,p33#871,p34#872,p35#873,p36#874,p37#875,p38#876,p39#877,link_name_p40#878,link_loc_name_p41#879,link_pg_loc_name_p42#880,p43#881,p44#882,p45#883,p46#884,p47#885,timestamp_p48#886,template_taxonomy_p49#887,template_name_p50#888,email_home_taxonomy_p51#889,editor_taxonomy_p52#890,kb_detail_p53#891,stock_image_name_p54#892,sent_template_name_p55#893,p56#894,p57#895,p58#896,p59#897,p60#898,p61#899,p62#900,p63#901,p64#902,p65#903,p66#904,p67#905,p68#906,p69#907,p70#908,p71#909,p72#910,p73#911,p74#912,p75#913,purchaseid#914,quarterly_visitor#915L,ref_domain#916,referrer#917,ref_type#918,resolution#919,sampled_hit#920,search_engine#921,search_page_num#922L,secondary_hit#923,service#924,sourceid#925L,s_resolution#926,state#927,stats_server#928,tnt#929,tnt_post_vista#930,transactionid#931,truncated_hit#932,t_time_info#933,ua_color#934,ua_os#935,ua_pixels#936,user_agent#937,user_hash#938L,userid#939L,username#940,user_server#941,va_closer_detail#942,va_closer_id#943L,va_finder_detail#944,va_finder_id#945L,va_instance_event#946,va_new_engagement#947,visid_high#948L,visid_low#949L,visid_new#950,visid_timestamp#951L,visid_type#952,visit_keywords#953,visit_num#954,visit_page_num#955,visit_referrer#956,visit_search_engine#957,visit_start_pagename#958,visit_start_page_url#959,visit_start_time_gmt#960L,weekly_visitor#961,yearly_visitor#962L,zip#963,browser_desc#964,color_desc#965,connection_type_desc#966,country_desc#967,event_list_desc#968,javascript_desc#969,language_desc#970,os_desc#971,plugins_desc#972,ref_type_desc#973,resolution_desc#974,search_engine_desc#975]  ConvertToSafe   TungstenProject [soid_e1#30,visid_high#460L,visid_low#461L,date_time#25L]    Filter (instr(event_list#105,202) > 0)     Scan ParquetRelation[hdfs://xxx/data/event_parquet/2015/08/21][accept_language#0,browser#1L,browser_height#2,browser_width#3,campaign#4,c_color#5,channel#6,click_action#7,click_action_type#8,click_context#9,click_context_type#10,click_sourceId#11L,click_tag#12,code_ver#13,color#14,connection_type#15,cookies#16,country#17,ct_connect_type#18,currency#19,curr_factor#20,curr_rate#21,cust_hit_time_gmt#22L,cust_visid#23,daily_visitor#24,date_time#25L,domain#26,duplicated_from#27,duplicate_events#28,duplicate_purchase#29,soid_e1#30,lpuid_e2#31,customer_status_e3#32,partner_uid_e4#33,ab_split_group_e5#34,application_e6#35,page_name_e7#36,originating_page_e8#37,email_signup_e9#38,survey_signup_e10#39,event_signup_e11#40,email_paying_e12#41,survey_paying_e13#42,event_paying_e14#43,image_hosting_paying_e15#44,archive_paying_e16#45,internal_search_query_e17#46,internal_search_page_e18#47,download_file_name_e19#48,download_file_page_e20#49,campaign_stacking_e21#50,improveit_category_e22#51,improveit_keyword_e23#52,omtr_vi_id_e24#53,aprimo_id_e25#54,link_name_e26#55,user_agent_e27#56,page_type_desc_e28#57,site_region_language_e29#58,social_campaigns_signup_e30#59,session_history_e31#60,external_data_source_type_e32#61,rmc_code_e33#62,video_name_e34#63,video_segment_e35#64,video_content_e36#65,free_user_prod_name_e37#66,web_voc_id_e38#67,ic_code_e39#68,hit_count_e40#69,signup_form_orig_page_e41#70,e42#71,e43#72,e44#73,pyament_plan_type_e45#74,purchased_product_name_e46#75,purchased_product_category_e47#76,link_loc_name_e48#77,link_pg_loc_name_e49#78,email_test_counter_e50#79,email_preview_counter_e51#80,email_send_counter_e52#81,time_emcf_tpicker_e53#82,time_emcf_editor_e54#83,time_emcf_scheduler_e55#84,time_emcf_build_e56#85,time_emcf_send_success_e57#86,timestamp_e58#87,templates_returned_e59#88,email_agent_id_e60#89,template_name_e61#90,template_hit_counter_e62#91,template_taxonomy_e63#92,email_home_taxonomy_e64#93,email_home_hit_counter_e65#94,editor_taxonomy_t66#95,editor_block_detail_e67#96,editor_block_title_e68#97,editor_hit_counter_e69#98,email_insert_detail_e70#99,email_creation_source_e71#100,e72#101,e73#102,e74#103,adlens_ef_id_e75#104,event_list#105,exclude_hit#106,first_hit_pagename#107,first_hit_page_url#108,first_hit_referrer#109,first_hit_time_gmt#110L,geo_city#111,geo_country#112,geo_dma#113,geo_region#114,geo_zip#115,hier1#116,hier2#117,hier3#118,hier4#119,hier5#120,hitid_high#121L,hitid_low#122L,hit_source#123,hit_time_gmt#124L,homepage#125,hourly_visitor#126,ip#127,ip2#128,java_enabled#129,javascript#130,j_jscript#131,language#132,last_hit_time_gmt#133L,last_purchase_num#134,last_purchase_time_gmt#135L,mobile_id#136L,monthly_visitor#137L,mvvar1#138,mvvar2#139,mvvar3#140,namespace#141,new_visit#142,os#143L,page_event#144,page_event_var1#145,page_event_var2#146,page_event_var3#147,page_name#148,page_type#149,page_url#150,paid_search#151,partner_plugins#152,persistent_cookie#153,plugins#154,post_browser_height#155,post_browser_width#156,post_campaign#157,post_channel#158,post_cookies#159,post_currency#160,post_cust_hit_time_gmt#161L,post_cust_visid#162,soid_pe1#163,lpuid_pe2#164,customer_status_pe3#165,partner_uid_pe4#166,ab_split_group_pe5#167,application_pe6#168,page_name_pe7#169,pe8#170,pe9#171,pe10#172,pe11#173,pe12#174,pe13#175,pe14#176,pe15#177,pe16#178,pe17#179,pe18#180,pe19#181,pe20#182,campaign_stacking_pe21#183,category_name_pe22#184,improvedit_keyword_pe23#185,omtr_vi_id_pe24#186,aprimo_id_pe25#187,link_name_pe26#188,pe27#189,pe28#190,site_region_language_pe29#191,pe30#192,session_history_pe31#193,pe32#194,rmc_code_pe33#195,video_name_pe34#196,pe35#197,pe36#198,pe37#199,pe38#200,ic_code_pe39#201,template_hit_crt_pe40#202,pe41#203,pe42#204,pe43#205,pe44#206,payment_plan_type_pe45#207,purchased_product_name_pe46#208,purchased_product_category_pe47#209,link_loc_name_pe48#210,link_pg_loc_name_pe49#211,email_test_ctr_pe50#212,email_preview_ctr_pe51#213,email_send_ctr_pe52#214,time_emcf_tpicker_pe53#215,time_emcf_editor_pe54#216,time_emcf_scheduler_pe55#217,time_emcf_build_pe56#218,time_emcf_send_success_pe57#219,pe58#220,pe59#221,email_agent_id_pe60#222,template_name_pe61#223,template_hit_ctr_pe62#224,template_taxonomy_pe63#225,email_home_taxonomy_pe64#226,email_home_hit_crt_pe65#227,editor_taxonomy_pe66#228,editor_block_detail_pe67#229,editor_block_title_pe68#230,editor_hit_crt_pe69#231,editor_insert_detail_pe70#232,email_creation_source_pe71#233,pe72#234,pe73#235,pe74#236,adlens_ef_id_pe75#237,post_event_list#238,post_hier1#239,post_hier2#240,post_hier3#241,post_hier4#242,post_hier5#243,post_java_enabled#244,post_keywords#245,post_mvvar1#246,post_mvvar2#247,post_mvvar3#248,post_page_event#249,post_page_event_var1#250,post_page_event_var2#251,post_page_event_var3#252,post_pagename#253,post_pagename_no_url#254,post_page_type#255,post_page_url#256,post_partner_plugins#257,post_persistent_cookie#258,post_product_list#259,soid_pp1#260,lpuid_pp2#261,custom_status_pp3#262,internal_external_pp4#263,ab_split_group_pp5#264,application_pp6#265,internal_search_query_pp7#266,pp8#267,pp9#268,pp10#269,pp11#270,sops_pp12#271,pp13#272,pp14#273,pp15#274,pp16#275,pp17#276,pp18#277,signup_form_pp19#278,pp20#279,pp21#280,category_name_pp22#281,improveit_keyword_pp23#282,pp24#283,sops_hero_name_pp25#284,video_name_pp26#285,pp27#286,topnav_link_name_pp28#287,pp29#288,pp30#289,pp31#290,pp32#291,pp33#292,pp34#293,pp35#294,pp36#295,pp37#296,pp38#297,pp39#298,link_name_pp40#299,link_loc_name_pp41#300,link_pg_loc_name_pp42#301,pp43#302,pp44#303,pp45#304,pp46#305,pp47#306,pp48#307,template_taxonomy_pp49#308,template_name_pp50#309,email_home_taxonomy_pp51#310,editor_taxonomy_pp52#311,kb_details_pp53#312,pp54#313,pp55#314,pp56#315,pp57#316,pp58#317,pp59#318,pp60#319,pp61#320,pp62#321,pp63#322,pp64#323,pp65#324,pp66#325,pp67#326,pp68#327,pp69#328,pp70#329,pp71#330,pp72#331,pp73#332,pp74#333,pp75#334,post_purchaseid#335,post_referrer#336,post_search_engine#337,post_state#338,post_survey#339,post_tnt#340,post_transactionid#341,post_t_time_info#342,post_visid_high#343L,post_visid_low#344L,post_visid_type#345,post_zip#346,p_plugins#347,prev_page#348L,product_list#349,product_merchandising#350,soid_p1#351,p2#352,p3#353,p4#354,p5#355,p6#356,p7#357,p8#358,p9#359,p10#360,p11#361,p12#362,p13#363,p14#364,p15#365,p16#366,p17#367,p18#368,signup_form_p19#369,p20#370,p21#371,improveit_category_p22#372,improveit_keyword_p23#373,test_drive_type_p24#374,sops_hero_name_p25#375,video_name_p26#376,previous_page_name_p27#377,topnav_link_name_p28#378,p29#379,marketplace_detail_p30#380,p31#381,p32#382,p33#383,p34#384,p35#385,p36#386,p37#387,p38#388,p39#389,link_name_p40#390,link_loc_name_p41#391,link_pg_loc_name_p42#392,p43#393,p44#394,p45#395,p46#396,p47#397,timestamp_p48#398,template_taxonomy_p49#399,template_name_p50#400,email_home_taxonomy_p51#401,editor_taxonomy_p52#402,kb_detail_p53#403,stock_image_name_p54#404,sent_template_name_p55#405,p56#406,p57#407,p58#408,p59#409,p60#410,p61#411,p62#412,p63#413,p64#414,p65#415,p66#416,p67#417,p68#418,p69#419,p70#420,p71#421,p72#422,p73#423,p74#424,p75#425,purchaseid#426,quarterly_visitor#427L,ref_domain#428,referrer#429,ref_type#430,resolution#431,sampled_hit#432,search_engine#433,search_page_num#434L,secondary_hit#435,service#436,sourceid#437L,s_resolution#438,state#439,stats_server#440,tnt#441,tnt_post_vista#442,transactionid#443,truncated_hit#444,t_time_info#445,ua_color#446,ua_os#447,ua_pixels#448,user_agent#449,user_hash#450L,userid#451L,username#452,user_server#453,va_closer_detail#454,va_closer_id#455L,va_finder_detail#456,va_finder_id#457L,va_instance_event#458,va_new_engagement#459,visid_high#460L,visid_low#461L,visid_new#462,visid_timestamp#463L,visid_type#464,visit_keywords#465,visit_num#466,visit_page_num#467,visit_referrer#468,visit_search_engine#469,visit_start_pagename#470,visit_start_page_url#471,visit_start_time_gmt#472L,weekly_visitor#473,yearly_visitor#474L,zip#475,browser_desc#476,color_desc#477,connection_type_desc#478,country_desc#479,event_list_desc#480,javascript_desc#481,language_desc#482,os_desc#483,plugins_desc#484,ref_type_desc#485,resolution_desc#486,search_engine_desc#487]