h2oai / sparkling-water

Sparkling Water provides H2O functionality inside Spark cluster
https://docs.h2o.ai/sparkling-water/3.3/latest-stable/doc/index.html
Apache License 2.0
967 stars 360 forks source link

conversion of sparse data DataFrame to H2OFrame is slow #4110

Closed exalate-issue-sync[bot] closed 1 year ago

exalate-issue-sync[bot] commented 1 year ago

it takes a long time to build the model on a sparse dataset (89x5000) when read in using parquet format on a 5 executor SW cluster.

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: I was just testing this on yarn cluster of size 5 started as

{code:bash} ./bin/sparkling-shell --num-executors 5 --executor-memory 5g --driver-memory 10g --conf "spark.hadoop.yarn.timeline-service.enabled=false" --conf "spark.scheduler.minRegisteredResourcesRatio=1" --conf "spark.sql.autoBroadcastJoinThreshold=-1" {code}

This is the code which I was testing:

{code:java} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.h2o._ val hc = H2OContext.getOrCreate(spark)

def time[R](block: => R): R = {
val t0 = System.nanoTime() val result = block // call-by-name val t1 = System.nanoTime() println("Elapsed time: " + (t1 - t0) + "ns") result }

val data = (1 to 89).map(num => ("lab_" + num, Vectors.sparse(50000, Seq((num, num.toDouble))))) val df = spark.sparkContext.parallelize(data, 10).toDF df.write.parquet("test.parquet") val parquetFileDF = spark.read.parquet("test.parquet")

time{ parquetFileDF.count() }

time{ val h2oframe = hc.asH2OFrame(parquetFileDF) } {code}

The data materialisation took less then 1 second, the conversion to h2o frame about 4.8 seconds. This was tested on data were there were just a single, non-zero value in the sparse vector

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: On data where each value in sparse vector was specified, it took roughly the same. ( no zero-values)

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: The same for the case where non-zero value is on every second position.

So tested cases:

All behaved very similar and took about 5 seconds to convert which is fine. I exported the parquet to local file-system.

Are they loading this parquet from hdfs or local file-system as well ?

But from my testing I can't reproduce this slowness, was their environment any special ?

CC: [~accountid:557058:5bcbac08-75cf-4c6b-b4d2-294f7c0fe9b8] [~accountid:557058:389d9607-5bd8-4611-8c6a-755fe9295223] [~accountid:557058:04659f86-fbfe-4d01-90c9-146c34df6ee6]

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: bad and good news! I can see that it is really slow from PySparkling.

This code is a few seconds in Scala

{code:java} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.h2o. val hc = H2OContext.getOrCreate(spark) val data = (1 to 89).map(num => ("lab" + num, Vectors.sparse(50000, Seq((num, num.toDouble))))) val df = spark.sparkContext.parallelize(data, 10).toDF val h2oframe = hc.asH2OFrame(df) {code}

This so far hasn’t finished in Python

{code:java} from pysparkling import from pyspark.mllib.linalg import hc = H2OContext.getOrCreate(spark) data = [(float(x), SparseVector(50000, {x: float(x)})) for x in range(1, 90)] df = sc.parallelize(data).toDF() hc.as_h2o_frame(df) {code}

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: Scala Job Spark UI !Screen Shot 2017-11-22 at 10.10.20.png|thumbnail!

Python Job Spark UI

!Screen Shot 2017-11-22 at 10.10.43.png|thumbnail!

So the tasks took around the same time.

So for some reason on the pysparkling side it takes longer time (but it’s not not in hours, but takes around 5 minutes ) but the task itself last the same

exalate-issue-sync[bot] commented 1 year ago

Jakub Hava commented: this problem is caused at

{code:java} @staticmethod def from_java_h2o_frame(h2o_frame, h2o_frame_id):

Cache Java reference to the backend frame

    sid = h2o_frame_id.toString()
    fr = H2OFrame.get_frame(sid)
    fr._java_frame = h2o_frame
    fr._java_frame_sid = sid
    fr._backed_by_java_obj = True
    return fr

{code}, particularly this bit takes long and {{fr = H2OFrame.get_frame(sid)}}

so {code:java} 2017-11-22 01:43:55,164 GET 200 200977 ms /3/Frames/frame_rdd_22_8b083d34c1e7262d408502770bb445eewhich is the this call res = h2o.api("GET /3/Frames/%s" % self._id, data={"row_count": rows})["frames"][0] {code}

And the root cause is {{doFetch}} method in FramesHandler since it needs to load first few rows to store into {{expr}} on python side. And since this is sparse data, these 10 rows is 50000*50 doubles which takes some times. Minor note, if we pass 0 to it, then this code makes it 100

{code:java} if( row_count == 0 ) row_count = 100; // 100 rows by default if( column_count == 0 ) column_count = f.numCols() - column_offset; // full width by default {code}

0 should be valid value, however and we do not have any option.

But rows are not the core issue, but columns. What we can do is to specify also default number of columns to show using {{column_coun}} ( it’s already in the API, but python is not using it). Right now it’s default which means all. This would help in the sparse data case

DinukaH2O commented 1 year ago

JIRA Issue Migration Info

Jira Issue: SW-602 Assignee: Jakub Hava Reporter: Nidhi Mehta State: Resolved Fix Version: 2.2.4 Attachments: Available (Count: 2) Development PRs: Available

Linked PRs from JIRA

https://github.com/h2oai/h2o-3/pull/1788 https://github.com/h2oai/h2o-3/pull/1808 https://github.com/h2oai/h2o-3/pull/1810 https://github.com/h2oai/sparkling-water/pull/477

Attachments From Jira

Attachment Name: Screen Shot 2017-11-22 at 10.10.20.png Attached By: Jakub Hava File Link:https://h2o-jira-github-migration.s3.amazonaws.com/Sparkling-Water/SW-602/Screen Shot 2017-11-22 at 10.10.20.png

Attachment Name: Screen Shot 2017-11-22 at 10.10.43.png Attached By: Jakub Hava File Link:https://h2o-jira-github-migration.s3.amazonaws.com/Sparkling-Water/SW-602/Screen Shot 2017-11-22 at 10.10.43.png

hasithjp commented 1 year ago

JIRA Issue Migration Info Cont'd

Jira Issue Created Date: 2017-11-16T09:53:01.226-0800