[ https://issues.apache.org/jira/browse/SPARK-27648?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16836244#comment-16836244 ]
tommy duan edited comment on SPARK-27648 at 5/9/19 9:55 AM: ------------------------------------------------------------ [~kabhwan] the print the progress information,as bellow: {code:java} Query made progress: { "id" : "2111635f-d22c-4154-bf50-6abb74e3bcc0", "runId" : "9e9e0599-81a7-42f0-8f55-bd0850f82374", "name" : "queryEutrancellHourAgg", "timestamp" : "2019-05-09T08:50:05.976Z", "batchId" : 1451, "numInputRows" : 22947, "inputRowsPerSecond" : 37.86783634995445, "processedRowsPerSecond" : 54.186482542351264, "durationMs" : { "addBatch" : 397693, "getBatch" : 0, "getEndOffset" : 1, "queryPlanning" : 22432, "setOffsetRange" : 3037, "triggerExecution" : 423482, "walCommit" : 115 }, "eventTime" : { "avg" : "2019-05-09T08:59:58.980Z", "max" : "2019-05-09T09:00:00.000Z", "min" : "2019-05-09T08:45:00.000Z", "watermark" : "2019-05-09T07:05:00.000Z" }, "stateOperators" : [ { "numRowsTotal" : 487118, "numRowsUpdated" : 22947, "memoryUsedBytes" : 11040408699, "customMetrics" : { "loadedMapCacheHitCount" : 2423996, "loadedMapCacheMissCount" : 900, "stateOnCurrentVersionSizeBytes" : 5954563251 } }, { "numRowsTotal" : 1647149, "numRowsUpdated" : 22947, "memoryUsedBytes" : 404924895, "customMetrics" : { "loadedMapCacheHitCount" : 1265391, "loadedMapCacheMissCount" : 900, "stateOnCurrentVersionSizeBytes" : 330813607 } } ], "sources" : [ { "description" : "KafkaV2[Subscribe[EUTRANCELL-Q1]]", "startOffset" : { "EUTRANCELL-Q1" : { "17" : 175360173, "8" : 174060651, "11" : 173139667, "2" : 174300640, "5" : 174804538, "14" : 177149516, "13" : 174998384, "4" : 175707317, "16" : 175120267, "7" : 175501102, "1" : 177652258, "10" : 175119811, "19" : 172012970, "18" : 173720382, "9" : 175036666, "3" : 173266983, "12" : 174741999, "15" : 173502850, "6" : 174331649, "0" : 173150031 } }, "endOffset" : { "MY-Q1" : { "17" : 175361333, "8" : 174061845, "11" : 173140851, "2" : 174301768, "5" : 174805693, "14" : 177150647, "13" : 174999529, "4" : 175708452, "16" : 175121423, "7" : 175502259, "1" : 177653355, "10" : 175120977, "19" : 172014087, "18" : 173721537, "9" : 175037859, "3" : 173268136, "12" : 174743133, "15" : 173503912, "6" : 174332803, "0" : 173151202 } }, "numInputRows" : 22947, "inputRowsPerSecond" : 37.86783634995445, "processedRowsPerSecond" : 54.186482542351264 } ], "sink" : { "description" : "org.apache.spark.sql.kafka010.KafkaSourceProvider@3de5060e" } }{code} The SPARK UI display as bellow: !image-2019-05-09-17-51-14-036.png! My concern is that when "Storage Memory" shows that it occupies more memory than "1.5G", Executor will have a memory overflow. In this case, what should I do? Please help me think about the solution. Thank you very much. was (Author: yy3b2007com): [~kabhwan] the print the progress information,as bellow: {code:java} Query made progress: { "id" : "2111635f-d22c-4154-bf50-6abb74e3bcc0", "runId" : "9e9e0599-81a7-42f0-8f55-bd0850f82374", "name" : "queryEutrancellHourAgg", "timestamp" : "2019-05-09T08:50:05.976Z", "batchId" : 1451, "numInputRows" : 22947, "inputRowsPerSecond" : 37.86783634995445, "processedRowsPerSecond" : 54.186482542351264, "durationMs" : { "addBatch" : 397693, "getBatch" : 0, "getEndOffset" : 1, "queryPlanning" : 22432, "setOffsetRange" : 3037, "triggerExecution" : 423482, "walCommit" : 115 }, "eventTime" : { "avg" : "2019-05-09T08:59:58.980Z", "max" : "2019-05-09T09:00:00.000Z", "min" : "2019-05-09T08:45:00.000Z", "watermark" : "2019-05-09T07:05:00.000Z" }, "stateOperators" : [ { "numRowsTotal" : 487118, "numRowsUpdated" : 22947, "memoryUsedBytes" : 11040408699, "customMetrics" : { "loadedMapCacheHitCount" : 2423996, "loadedMapCacheMissCount" : 900, "stateOnCurrentVersionSizeBytes" : 5954563251 } }, { "numRowsTotal" : 1647149, "numRowsUpdated" : 22947, "memoryUsedBytes" : 404924895, "customMetrics" : { "loadedMapCacheHitCount" : 1265391, "loadedMapCacheMissCount" : 900, "stateOnCurrentVersionSizeBytes" : 330813607 } } ], "sources" : [ { "description" : "KafkaV2[Subscribe[EUTRANCELL-Q1]]", "startOffset" : { "EUTRANCELL-Q1" : { "17" : 175360173, "8" : 174060651, "11" : 173139667, "2" : 174300640, "5" : 174804538, "14" : 177149516, "13" : 174998384, "4" : 175707317, "16" : 175120267, "7" : 175501102, "1" : 177652258, "10" : 175119811, "19" : 172012970, "18" : 173720382, "9" : 175036666, "3" : 173266983, "12" : 174741999, "15" : 173502850, "6" : 174331649, "0" : 173150031 } }, "endOffset" : { "MY-Q1" : { "17" : 175361333, "8" : 174061845, "11" : 173140851, "2" : 174301768, "5" : 174805693, "14" : 177150647, "13" : 174999529, "4" : 175708452, "16" : 175121423, "7" : 175502259, "1" : 177653355, "10" : 175120977, "19" : 172014087, "18" : 173721537, "9" : 175037859, "3" : 173268136, "12" : 174743133, "15" : 173503912, "6" : 174332803, "0" : 173151202 } }, "numInputRows" : 22947, "inputRowsPerSecond" : 37.86783634995445, "processedRowsPerSecond" : 54.186482542351264 } ], "sink" : { "description" : "org.apache.spark.sql.kafka010.KafkaSourceProvider@3de5060e" } }{code} > In Spark2.4 Structured Streaming:The executor storage memory increasing over > time > --------------------------------------------------------------------------------- > > Key: SPARK-27648 > URL: https://issues.apache.org/jira/browse/SPARK-27648 > Project: Spark > Issue Type: Bug > Components: Structured Streaming > Affects Versions: 2.4.0 > Reporter: tommy duan > Priority: Major > Attachments: image-2019-05-09-17-51-14-036.png > > > *Spark Program Code Business:* > Read the topic on kafka, aggregate the stream data sources, and then output > it to another topic line of kafka. > *Problem Description:* > *1) Using spark structured streaming in CDH environment (spark 2.2)*, memory > overflow problems often occur (because of too many versions of state stored > in memory, this bug has been modified in spark 2.4). > {code:java} > /spark-submit \ > --conf “spark.yarn.executor.memoryOverhead=4096M” > --num-executors 15 \ > --executor-memory 3G \ > --executor-cores 2 \ > --driver-memory 6G \{code} > {code} > Executor memory exceptions occur when running with this submit resource under > SPARK 2.2 and the normal running time does not exceed one day. > The solution is to set the executor memory larger than before > {code:java} > My spark-submit script is as follows: > /spark-submit\ > conf "spark. yarn. executor. memoryOverhead = 4096M" > num-executors 15\ > executor-memory 46G\ > executor-cores 3\ > driver-memory 6G\ > ...{code} > In this case, the spark program can be guaranteed to run stably for a long > time, and the executor storage memory is less than 10M (it has been running > stably for more than 20 days). > *2) From the upgrade information of Spark 2.4, we can see that the problem of > large memory consumption of state storage has been solved in Spark 2.4.* > So we upgraded spark to SPARK 2.4 under CDH, tried to run the spark program, > and found that the use of memory was reduced. > But a problem arises, as the running time increases, the storage memory of > executor is growing (see Executors - > Storage Memory from the Spark on Yarn > Resource Manager UI). > This program has been running for 14 days (under SPARK 2.2, running with > this submit resource, the normal running time is not more than one day, > Executor memory abnormalities will occur). > The script submitted by the program under spark2.4 is as follows: > {code:java} > /spark-submit \ > --conf “spark.yarn.executor.memoryOverhead=4096M” > --num-executors 15 \ > --executor-memory 3G \ > --executor-cores 2 \ > --driver-memory 6G > {code} > Under Spark 2.4, I counted the size of executor memory as time went by during > the running of the spark program: > |Run-time(hour)|Storage Memory size(MB)|Memory growth rate(MB/hour)| > |23.5H|41.6MB/1.5GB|1.770212766| > |108.4H|460.2MB/1.5GB|4.245387454| > |131.7H|559.1MB/1.5GB|4.245254366| > |135.4H|575MB/1.5GB|4.246676514| > |153.6H|641.2MB/1.5GB|4.174479167| > |219H|888.1MB/1.5GB|4.055251142| > |263H|1126.4MB/1.5GB|4.282889734| > |309H|1228.8MB/1.5GB|3.976699029| -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org