Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-9058

if set `spark.sql.codegen` is true,More than 100 aggregation operation, it exceeds JVM code size limits

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Duplicate
    • 1.4.0
    • 1.5.0
    • SQL
    • None

    Description

      if set “spark.sql.codegen”is true,there are more then 100 aggregation operation,it will wrong.
      The following error arise:

          
      15/07/15 09:34:51 ERROR executor.Executor: Exception in task 0.0 in stage 251.0 (TID 108305) 
      java.util.concurrent.ExecutionException: scala.tools.reflect.ToolBoxError: reflective compilation has failed: 
      
      Could not write class __wrapper$99$8a7b0d21732141bb96e5931945ce95f9/__wrapper$99$8a7b0d21732141bb96e5931945ce95f9$$anonfun$wrapper$1$$anon$1 because it exceeds JVM code size limits. Method apply's code too large! 
      at org.spark-project.guava.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:306) 
      at org.spark-project.guava.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:293) 
      at org.spark-project.guava.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
      at org.spark-project.guava.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:135) 
      at org.spark-project.guava.cache.LocalCache$Segment.getAndRecordStats(LocalCache.java:2410) 
      at org.spark-project.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2380) 
      at org.spark-project.guava.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2342) 
      at org.spark-project.guava.cache.LocalCache$Segment.get(LocalCache.java:2257) 
      at org.spark-project.guava.cache.LocalCache.get(LocalCache.java:4000) 
      at org.spark-project.guava.cache.LocalCache.getOrLoad(LocalCache.java:4004) 
      at org.spark-project.guava.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) 
      at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:105) 
      at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:102) 
      at org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:170) 
      at org.apache.spark.sql.execution.GeneratedAggregate$$anonfun$9.apply(GeneratedAggregate.scala:261) 
      at org.apache.spark.sql.execution.GeneratedAggregate$$anonfun$9.apply(GeneratedAggregate.scala:246) 
      at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$17.apply(RDD.scala:686) 
      at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$17.apply(RDD.scala:686) 
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) 
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) 
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) 
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) 
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) 
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) 
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) 
      at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) 
      at org.apache.spark.rdd.RDD.iterator(RDD.scala:244) 
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63) 
      at org.apache.spark.scheduler.Task.run(Task.scala:70) 
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213) 
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
      at java.lang.Thread.run(Thread.java:745) 
      Caused by: scala.tools.reflect.ToolBoxError: reflective compilation has failed: 
      
      Could not write class __wrapper$99$8a7b0d21732141bb96e5931945ce95f9/__wrapper$99$8a7b0d21732141bb96e5931945ce95f9$$anonfun$wrapper$1$$anon$1 because it exceeds JVM code size limits. Method apply's code too large! 
      at scala.tools.reflect.ToolBoxFactory$ToolBoxImpl$ToolBoxGlobal.throwIfErrors(ToolBoxFactory.scala:315) 
      at scala.tools.reflect.ToolBoxFactory$ToolBoxImpl$ToolBoxGlobal.compile(ToolBoxFactory.scala:249) 
      at scala.tools.reflect.ToolBoxFactory$ToolBoxImpl.compile(ToolBoxFactory.scala:416) 
      at scala.tools.reflect.ToolBoxFactory$ToolBoxImpl.eval(ToolBoxFactory.scala:419) 
      at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.create(GenerateMutableProjection.scala:74)
      at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.create(GenerateMutableProjection.scala:26)
      at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:92) 
      at org.spark-project.guava.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3599) 
      at org.spark-project.guava.cache.LocalCache$Segment.loadSync(LocalCache.java:2379) 
      ... 27 more
      

      The query is:

      select 
      a.user_id, 
      a.phone_no, 
      sum(case when a.total_fee>0 then 1 else 0 end) as bill_num, 
      sum(1) as call_num, 
      sum(a.call_dur_secnd_num) as CALL_DUR, 
      sum(a.loc_call_fee) as BASIC_FEE, 
      sum(a.roam_fee) as ROAM_CALL_FEE, 
      sum(case when a.shot_no_flag='0' then a.call_dur_secnd_num else 0 end) as VPMN_IN_CALL_DUR, 
      sum(case when a.shot_no_flag='0' then 1 else 0 end) as VPMN_IN_CALL_NUM, 
      sum(case when a.shot_no_flag='0' then a.total_fee else 0 end) as VPMN_IN_CAL_FEE, 
      sum(case when a.shot_no_flag='0' then 0 else a.call_dur_secnd_num end) as VPMN_OUT_CALL_DUR, 
      sum(case when a.shot_no_flag='0' then 0 else 1 end) as VPMN_OUT_CALL_NUM, 
      sum(case when a.shot_no_flag='0' then 0 else a.total_fee end) as VPMN_OUT_FEE, 
      sum(a.toll_fee+a.toll_fee_2+a.toll_append_fee) as TOLL_CALL_FEE, 
      sum(case when a.busy_idle_flag ='10' then a.call_dur_secnd_num else 0 end) as BUSY_CALL_DUR, 
      sum(case when a.busy_idle_flag ='10' then 1 else 0 end)as BUSY_CALL_NUM, 
      sum(case when a.busy_idle_flag ='10' then a.total_fee else 0 end)as BUSY_CALL_FEE, 
      sum(case when a.busy_idle_flag ='11' then a.call_dur_secnd_num else 0 end) as IDLE_CALL_DUR, 
      sum(case when a.busy_idle_flag ='11' then 1 else 0 end)as IDLE_CALL_NUM, 
      sum(case when a.busy_idle_flag ='11' then a.total_fee else 0 end)as IDLE_CALL_FEE, 
      sum(case when a.call_type ='10' then 1 else 0 end) as CALING_NUM, 
      sum(case when a.call_type ='10' then a.call_dur_secnd_num else 0 end) as CALING_CALL_DUR, 
      sum(case when a.call_type ='10' then a.bill_dur else 0 end) as CALING_BILL_DUR, 
      sum(case when a.call_type ='10' then a.total_fee else 0 end) as CALING_FEE, 
      sum(case when a.call_type ='11' then 1 else 0 end) as CALLED_NUM, 
      sum(case when a.call_type ='11' then a.call_dur_secnd_num else 0 end) as CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' then a.bill_dur else 0 end) as CALLED_BILL_DUR, 
      sum(case when a.call_type ='11' then a.total_fee else 0 end) as CALLED_FEE, 
      sum(case when a.call_type ='12' then 1 else 0 end) as FORW_NUM, 
      sum(case when a.call_type ='12' then a.call_dur_secnd_num else 0 end) as FORW_CALL_DUR, 
      sum(case when a.call_type ='12' then a.bill_dur else 0 end) as FORW_BILL_DUR, 
      sum(case when a.call_type ='12' then a.total_fee else 0 end) as FORW_FEE, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CMC' then 1 else 0 end) as FORW_CMC_NUM, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CMC' then a.call_dur_secnd_num else 0 end) as FORW_CMC_CALL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CMC' then a.bill_dur else 0 end) as FORW_CMC_BILL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_CDMA' then 1 else 0 end) as FORW_CDMA_NUM, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_CDMA' then a.call_dur_secnd_num else 0 end) as FORW_CDMA_CALL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_CDMA' then a.bill_dur else 0 end) as FORW_CDMA_BILL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_PSTN' then 1 else 0 end) as FORW_CTC_FIX_NUM, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_PSTN' then a.call_dur_secnd_num else 0 end) as FORW_CTC_FIX_CALL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_PSTN' then a.bill_dur else 0 end) as FORW_CTC_FIX_BILL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CUC' then 1 else 0 end) as FORW_CUC_NUM, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CUC' then a.call_dur_secnd_num else 0 end) as FORW_CUC_CALL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CUC' then a.bill_dur else 0 end) as FORW_CUC_BILL_DUR, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then 1 else 0 end) as INNET_CALL_NUM, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then a.call_dur_secnd_num else 0 end) as INNET_CALL_DUR, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then a.bill_dur else 0 end) as INNET_BILL_DUR, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then 0 else 1 end) as NETO_CALL_NUM, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then 0 else a.call_dur_secnd_num end) as NETO_CALL_DUR, 
      sum(case when a.spec_type = 'CMC_SERVICE_NO' then 1 else 0 end) as CMC_CSR_NUM, 
      sum(case when a.spec_type = 'CUC_SERVICE_NO' then 1 else 0 end) as CUC_CSR_NUM, 
      sum(case when a.spec_type = 'CTC_SERVICE_NO' then 1 else 0 end) as CTC_CSR_NUM, 
      sum(case when a.sort_name= 'OPP_CARR_CMC' then 0 else a.bill_dur end) as NETO_BILL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN') then 1 else 0 end) as CTC_CALING_NUM, 
      sum(case when a.call_type ='10' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN') then a.call_dur_secnd_num else 0 end) as CTC_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN') then a.bill_dur else 0 end) as CTC_CALING_BILL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CUC' then 1 else 0 end) as CUC_CALING_NUM, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CUC' then a.call_dur_secnd_num else 0 end) as CUC_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CUC' then bill_dur else 0 end) as CUC_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN')then 1 else 0 end) as CTC_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN') then a.call_dur_secnd_num else 0 end) as CTC_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name in ('OPP_CARR_CTC_CDMA','OPP_CARR_CTC_PSTN') then a.bill_dur else 0 end) as CTC_CALLED_BILL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CUC' then 1 else 0 end) as CUC_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CUC' then a.call_dur_secnd_num else 0 end) as CUC_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CUC' then a.bill_dur else 0 end) as CUC_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE in ('10','11','12','13','14') then 1 else 0 end ) as LCALING_CALL_NUM, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE in ('10','11','12','13','14') then a.call_dur_secnd_num else 0 end ) as LCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE in ('10','11','12','13','14') then a.bill_dur else 0 end ) as LCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE in ('10','11','12','13','14') then 1 else 0 end ) as LCALLED_CALL_NUM, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE in ('10','11','12','13','14') then a.call_dur_secnd_num else 0 end ) as LCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE in ('10','11','12','13','14') then a.bill_dur else 0 end ) as LCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE not in ('10','11','12','13','14') then 1 else 0 end ) as TOLL_CALING_NUM, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.call_dur_secnd_num else 0 end ) as TOLL_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.bill_dur else 0 end ) as TOLL_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then 1 else 0 end ) as TOLL_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.call_dur_secnd_num else 0 end ) as TOLL_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.bill_dur else 0 end ) as TOLL_CALLED_BILL_DUR, 
      sum(case when a.dial_type='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then 1 else 0 end ) as IP_TOLL_CALL_NUM, 
      sum(case when a.dial_type='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.call_dur_secnd_num else 0 end ) as IP_TOLL_CALL_DUR, 
      sum(case when a.dial_type='11' and a.TOLL_TYPE not in ('10','11','12','13','14') then a.bill_dur else 0 end ) as IP_TOLL_BILL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type='15' then 1 else 0 end) as PROV_IN_CALING_NUM, 
      sum(case when a.call_type ='10' and a.toll_type='15' then a.call_dur_secnd_num else 0 end) as PROV_IN_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type='15' then a.bill_dur else 0 end) as PROV_IN_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type='15' then 1 else 0 end) as PROV_IN_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.toll_type='15' then a.call_dur_secnd_num else 0 end) as PROV_IN_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type='15' then a.bill_dur else 0 end) as PROV_IN_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type='16' then 1 else 0 end) as LAND_CALING_NUM, 
      sum(case when a.call_type ='10' and a.toll_type='16' then call_dur_secnd_num else 0 end) as LAND_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type='16' then bill_dur else 0 end) as LAND_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type='16' then 1 else 0 end) as LAND_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.toll_type='16' then call_dur_secnd_num else 0 end) as LAND_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type='16' then bill_dur else 0 end) as LAND_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type in ('17','21','22') then 1 else 0 end) as INTER_CALING_NUM, 
      sum(case when a.call_type ='10' and a.toll_type in ('17','21','22') then a.call_dur_secnd_num else 0 end) as INTER_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type in ('17','21','22') then a.bill_dur else 0 end) as INTER_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type in ('17','21','22') then 1 else 0 end) as INTER_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.toll_type in ('17','21','22') then a.call_dur_secnd_num else 0 end) as INTER_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type in ('17','21','22') then a.bill_dur else 0 end) as INTER_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type in ('18','19','20') then 1 else 0 end) as HMT_CALING_NUM, 
      sum(case when a.call_type ='10' and a.toll_type in ('18','19','20') then a.call_dur_secnd_num else 0 end) as HMT_CALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.toll_type in ('18','19','20') then a.bill_dur else 0 end) as HMT_CALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type in ('18','19','20') then 1 else 0 end) as HMT_CALLED_NUM, 
      sum(case when a.call_type ='11' and a.toll_type in ('18','19','20') then a.call_dur_secnd_num else 0 end) as HMT_CALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.toll_type in ('18','19','20') then a.bill_dur else 0 end) as HMT_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('10','11','13') then 1 else 0 end ) as NO_RCALING_NUM, 
      sum(case when a.call_type ='10' and a.roam_type in ('10','11','13') then a.call_dur_secnd_num else 0 end ) as NO_RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('10','11','13') then a.bill_dur else 0 end ) as NO_RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('10','11','13') then 1 else 0 end ) as NO_RCALLED_NUM, 
      sum(case when a.call_type ='11' and a.roam_type in ('10','11','13') then a.call_dur_secnd_num else 0 end ) as NO_RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('10','11','13') then a.bill_dur else 0 end ) as NO_RCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type not in ('10','11','13') then 1 else 0 end ) as RCALING_NUM, 
      sum(case when a.call_type ='10' and a.roam_type not in ('10','11','13') then a.call_dur_secnd_num else 0 end ) as RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type not in ('10','11','13') then a.bill_dur else 0 end ) as RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type not in ('10','11','13') then 1 else 0 end ) as RCALLED_NUM, 
      sum(case when a.call_type ='11' and a.roam_type not in ('10','11','13') then a.call_dur_secnd_num else 0 end ) as RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type not in ('10','11','13') then a.bill_dur else 0 end ) as RCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type ='14' then 1 else 0 end ) as PROV_IN_RCALING_NUM, 
      sum(case when a.call_type ='10' and a.roam_type ='14' then a.call_dur_secnd_num else 0 end ) as PROV_IN_RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type ='14' then a.bill_dur else 0 end ) as PROV_IN_RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type ='14' then 1 else 0 end ) as PROV_IN_RCALLED_NUM, 
      sum(case when a.call_type ='11' and a.roam_type ='14' then a.call_dur_secnd_num else 0 end ) as PROV_IN_RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type ='14' then a.bill_dur else 0 end ) as PROV_IN_RCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type ='06' then 1 else 0 end ) as LAND_RCALING_NUM, 
      sum(case when a.call_type ='10' and a.roam_type ='06' then a.call_dur_secnd_num else 0 end ) as LAND_RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type ='06' then a.bill_dur else 0 end ) as LAND_RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type ='06' then 1 else 0 end ) as LAND_RCALLED_NUM, 
      sum(case when a.call_type ='11' and a.roam_type ='06' then a.call_dur_secnd_num else 0 end ) as LAND_RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type ='06' then a.bill_dur else 0 end ) as LAND_RCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('18','22') then 1 else 0 end ) as INTER_ROAM_CALING_NUM, 
      sum(case when a.call_type ='10' and a.roam_type in ('18','22') then a.call_dur_secnd_num else 0 end ) as INTER_RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('18','22') then a.bill_dur else 0 end ) as INTER_RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('18','22') then 1 else 0 end ) as INTER_RCALLED_NUM, 
      sum(case when a.call_type ='11' and a.roam_type in ('18','22') then a.call_dur_secnd_num else 0 end ) as INTER_RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('18','22') then a.bill_dur else 0 end ) as INTER_RCALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('19','20','21') then 1 else 0 end ) as HMT_RCALING_CALL_NUM, 
      sum(case when a.call_type ='10' and a.roam_type in ('19','20','21') then a.call_dur_secnd_num else 0 end ) as HMT_RCALING_CALL_DUR, 
      sum(case when a.call_type ='10' and a.roam_type in ('19','20','21') then a.bill_dur else 0 end ) as HMT_RCALING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('19','20','21') then 1 else 0 end ) as HMT_RCALLED_CALL_NUM, 
      sum(case when a.call_type ='11' and a.roam_type in ('19','20','21') then a.call_dur_secnd_num else 0 end ) as HMT_RCALLED_CALL_DUR, 
      sum(case when a.call_type ='11' and a.roam_type in ('19','20','21') then a.bill_dur else 0 end ) as HMT_RCALLED_BILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') then 1 else 0 end ) as LCALL_NUM, 
      sum(case when a.roam_type in ('10','11','13') then a.call_dur_secnd_num else 0 end ) as LCALL_DUR, 
      sum(case when a.roam_type in ('10','11','13') then a.bill_dur else 0 end ) as LBILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.dial_type='11' then a.bill_dur else 0 end ) as NO_RIP_BILL_DUR, 
      sum(case when a.roam_type not in ('10','11','13') and a.dial_type='11' then a.bill_dur else 0 end ) as ROAM_IP_BILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.TOLL_TYPE in ('10','11','12','13','14') then a.bill_dur else 0 end ) as NO_RLBILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.toll_type in ('17','21','22') then a.bill_dur else 0 end ) as NO_RINTER_BILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.toll_type in ('18','19','20') then a.bill_dur else 0 end ) as NO_RHMT_BILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.toll_type in ('16') then a.bill_dur else 0 end ) as NO_RLAND_BILL_DUR, 
      sum(case when a.roam_type in ('10','11','13') and a.toll_type in ('15') then a.bill_dur else 0 end ) as NO_RPROV_IN_BILL_DUR, 
      sum(case when a.dial_type='11' and a.toll_type in ('17','21','22') then a.bill_dur else 0 end ) as INTER_IP_BILL_DUR, 
      sum(case when a.dial_type='11' and a.toll_type in ('18','19','20') then a.bill_dur else 0 end ) as HMT_IP_BILL_DUR, 
      sum(case when a.roam_type not in ('10','11','13') and a.call_type ='10' and a.toll_type in ('17','21','22') then a.bill_dur else 0 end ) as RCALLING_INTER_BILL_DUR, 
      sum(case when a.roam_type not in ('10','11','13') and a.call_type ='10' and a.toll_type in ('18','19','20') then a.bill_dur else 0 end ) as RCALLING_HMT_BILL_DUR, 
      sum(case when a.roam_type not in ('10','11','13') and a.call_type ='10' and a.toll_type in ('16') then a.bill_dur else 0 end ) as RCALLING_LAND_BILL_DUR, 
      sum(case when a.roam_type not in ('10','11','13') and a.call_type ='10' and a.toll_type in ('15') then a.bill_dur else 0 end ) as RCALLING_PROV_IN_BILL_DUR, 
      sum(case when a.dial_type='11' and a.toll_type in ('16') then a.bill_dur else 0 end ) as LAND_IP_BILL_DUR, 
      sum(case when a.dial_type='11' and a.toll_type in ('15') then a.bill_dur else 0 end ) as PROV_IN_IP_BILL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CMC' then a.bill_dur else 0 end ) as INNET_CALLED_BILL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CMC' then a.bill_dur else 0 end ) as INNET_CALLING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CTC_CDMA' then a.bill_dur else 0 end ) as CDMA_CALLED_BILL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_CDMA' then a.bill_dur else 0 end ) as CDMA_FW_BILL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CTC_CDMA' then a.bill_dur else 0 end ) as CDMA_CALLING_BILL_DUR, 
      sum(case when a.call_type ='11' and a.sort_name= 'OPP_CARR_CTC_PSTN' then a.bill_dur else 0 end ) as PSTN_CALLED_BILL_DUR, 
      sum(case when a.call_type ='12' and a.sort_name= 'OPP_CARR_CTC_PSTN' then a.bill_dur else 0 end ) as PSTN_FW_BILL_DUR, 
      sum(case when a.call_type ='10' and a.sort_name= 'OPP_CARR_CTC_PSTN' then a.bill_dur else 0 end ) as PSTN_CALLING_BILL_DUR, 
      sum(case when a.call_type ='12' and a.opp_carr = '12' then a.bill_dur else 0 end ) as UNION_GSM_FW_BILL_DUR 
      from (select * from sumtest limit 1) a 
      group by a.user_id,a.phone_no;
      

      The table:

      CREATE TABLE `sumtest`( 
      `user_id` string, 
      `call_type` string, 
      `roam_type` string, 
      `toll_type` string, 
      `dial_type` string, 
      `phone_no` string, 
      `imei` string, 
      `dynm_roam_no` string, 
      `bts_id` string, 
      `lac_id` string, 
      `roam_region_no` string, 
      `opp_no` string, 
      `opp_carr` string, 
      `opp_net_type` string, 
      `opp_visit_region_no` string, 
      `opp_bts_id` string, 
      `opp_lac_code` string, 
      `opp_city_id` string, 
      `begin_time` string, 
      `call_dur_secnd_num` bigint, 
      `bill_dur` bigint, 
      `net_type` string, 
      `busy_idle_flag` smallint, 
      `shot_no_flag` smallint, 
      `total_fee` double, 
      `recycle_flag` smallint, 
      `cdr_busi_type` string, 
      `imsi` string, 
      `device_id` string, 
      `serv_type` string, 
      `serv_code` string, 
      `carr_id` string, 
      `trans_serv_suppr` string, 
      `cdr_split_flag` smallint, 
      `vpmn_call_type` string, 
      `trd_no` string, 
      `err_code` string, 
      `device_type` string, 
      `disc_track` string, 
      `cdr_sn` string, 
      `callreferenceno` string, 
      `deal_time` string, 
      `loc_call_fee` double, 
      `std_loc_call_fee` double, 
      `roam_fee` double, 
      `std_roam_fee` double, 
      `toll_fee` double, 
      `std_toll_fee` double, 
      `toll_append_fee` double, 
      `toll_fee_2` double, 
      `stat_month` string, 
      `stat_date` string, 
      `sort_name` string, 
      `spec_type` string) 
      ROW FORMAT SERDE 
      'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
      STORED AS INPUTFORMAT 
      'org.apache.hadoop.mapred.TextInputFormat' 
      OUTPUTFORMAT 
      'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
      

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              wangxj8 XiaoJing wang
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - 0.2h
                  0.2h
                  Remaining:
                  Remaining Estimate - 0.2h
                  0.2h
                  Logged:
                  Time Spent - Not Specified
                  Not Specified