Also, if the data isn't confidential, would you mind to send me a
compressed copy (don't cc user@spark.apache.org)?
Cheng
On 6/10/15 4:23 PM, 姜超才 wrote:
Hi Lian,
Thanks for your quick response.
I forgot mention that I have tuned driver memory from 2G to 4G, seems
got minor improvement, The dead way when fetching 1,400,000 rows
changed from "OOM::GC overhead limit exceeded" to " lost worker
heartbeat after 120s".
I will try to set "spark.sql.thriftServer.incrementalCollect" and
continue increase driver memory to 7G, and will send the result to you.
Thanks,
SuperJ
--------- 原始邮件信息 ---------
*发件人:* "Cheng Lian" <l...@databricks.com>
*收件人:* "Hester wang" <hester9...@gmail.com>, <user@spark.apache.org>
*主题:* Re: Met OOM when fetching more than 1,000,000 rows.
*日期:* 2015/06/10 16:15:47 (Wed)
Hi Xiaohan,
Would you please try to set
"spark.sql.thriftServer.incrementalCollect" to "true" and increasing
driver memory size? In this way, HiveThriftServer2 uses
RDD.toLocalIterator rather than RDD.collect().iterator to return the
result set. The key difference is that RDD.toLocalIterator retrieves a
single partition at a time, thus avoid holding the whole result set on
driver side. The memory issue happens on driver side rather than
executor side, so tuning executor memory size doesn't help.
Cheng
On 6/10/15 3:46 PM, Hester wang wrote:
Hi Lian,
I met a SparkSQL problem. I really appreciate it if you could give me
some help! Below is the detailed description of the problem, for more
information, attached are the original code and the log that you may
need.
Problem:
I want to query my table which stored in Hive through the SparkSQL
JDBC interface.
And want to fetch more than 1,000,000 rows. But met OOM.
sql = "select * from TEMP_ADMIN_150601_000001 limit XXX ";
My Env:
5 Nodes = One master + 4 workers, 1000M Network Switch , Redhat 6.5
Each node: 8G RAM, 500G Harddisk
Java 1.6, Scala 2.10.4, Hadoop 2.6, Spark 1.3.0, Hive 0.13
Data:
A table with user and there charge for electricity data.
About 1,600,000 Rows. About 28MB.
Each row occupy about 18 Bytes.
2 columns: user_id String, total_num Double
Repro Steps:
1. Start Spark
2. Start SparkSQL thriftserver, command:
/usr/local/spark/spark-1.3.0/sbin/start-thriftserver.sh --master
spark://cx-spark-001:7077 --conf spark.executor.memory=4g --conf
spark.driver.memory=2g --conf spark.shuffle.consolidateFiles=true
--conf spark.shuffle.manager=sort --conf
"spark.executor.extraJavaOptions=-XX:-UseGCOverheadLimit" --conf
spark.file.transferTo=false --conf spark.akka.timeout=2000 --conf
spark.storage.memoryFraction=0.4 --conf spark.cores.max=8 --conf
spark.kryoserializer.buffer.mb=256 --conf
spark.serializer=org.apache.spark.serializer.KryoSerializer --conf
spark.akka.frameSize=512 --driver-class-path
/usr/local/hive/lib/classes12.jar
3. Run the test code, see it in attached file: testHiveJDBC.java
4. Get the OOM:GC overhead limit exceeded or OOM: java heap space
or lost worker heartbeat after 120s. see the attached logs.
Preliminary diagnose:
1. When fetching less than 1,000,000 rows , it always success.
2. When fetching more than 1,300,000 rows , it always fail with OOM:
GC overhead limit exceeded.
3. When fetching about 1,040,000-1,200,000 rows, if query right after
the thrift server start up, most times success. if I successfully
query once then retry the same query, it will fail.
4. There are 3 dead pattern: OOM:GC overhead limit exceeded or OOM:
java heap space or lost worker heartbeat after 120s.
5. I tried to start thrift with different configure, give the worker
4G MEM or 2G MEM , got the same behavior. That means , no matter the
total MEM of worker, i can get less than 1,000,000 rows, and can not
get more than 1,300,000 rows.
Preliminary conclusions:
1. The total data is less than 30MB, It is so small, And there is no
complex computation operation.
So the failure is not caused by excessive memory requirements.
So I guess there are some defect in spark sql code.
2. Allocate 2G or 4G MEM to each worker, got same behavior.
This point strengthen my doubts: there are some defect in code. But I
can't find the specific location.
Thank you so much!
Best,
Xiaohan Wang