Re: AccumulatorV2 vs AccumulableParam (V1)

2018-05-03 Thread Wenchen Fan
Hi Sergey, Thanks for your valuable feedback! For 1: yea this is definitely a bug and I have sent a PR to fix it. For 2: I have left my comments on the JIRA ticket. For 3: I don't quite understand it, can you give some concrete examples? For 4: yea this is a problem, but I think it's not a big de

Re: Custom datasource as a wrapper for existing ones?

2018-05-03 Thread Wenchen Fan
Hi Jakub, Yea I think data source would be the most elegant way to solve your problem. Unfortunately in Spark 2.3 the only stable data source API is data source v1, which can't be used to implement high-performance data source. Data source v2 is still a preview version in Spark 2.3 and may change

Re: Custom datasource as a wrapper for existing ones?

2018-05-03 Thread Jakub Wozniak
Hi Wenchen, Thanks for your reply! We will have a look at the FileFormat. Actually looking at the V2 APIs I still don’t see how you can use the existing datasource (like Parquet + Hbase) and wrap it up in another one. Imagine you would like to load some files from parquet and load some tables f

Re: Custom datasource as a wrapper for existing ones?

2018-05-03 Thread Jörn Franke
It changed from 2.0 to 2.1 to 2.2 ... Not much but still changed. I somehow agree that this is still manageable > On 3. May 2018, at 16:46, Wenchen Fan wrote: > > Hi Jakub, > > Yea I think data source would be the most elegant way to solve your problem. > Unfortunately in Spark 2.3 the only s

Re: SparkR test failures in PR builder

2018-05-03 Thread Felix Cheung
This is resolved. Please see https://issues.apache.org/jira/browse/SPARK-24152 From: Kazuaki Ishizaki Sent: Wednesday, May 2, 2018 4:51:11 PM To: dev Cc: Joseph Bradley; Hossein Falaki Subject: Re: SparkR test failures in PR builder I am not familiar with SparkR

Re: SparkR test failures in PR builder

2018-05-03 Thread Xiao Li
Thank you for working on this. It helps a lot! Xiao 2018-05-03 8:42 GMT-07:00 Felix Cheung : > This is resolved. > > Please see https://issues.apache.org/jira/browse/SPARK-24152 > > -- > *From:* Kazuaki Ishizaki > *Sent:* Wednesday, May 2, 2018 4:51:11 PM > *To:* dev

[Structured streaming, V2] commit on ContinuousReader

2018-05-03 Thread Jiří Syrový
Version: 2.3, DataSourceV2, ContinuousReader Hi, We're creating a new data source to fetch data from streaming source that requires commiting received data and we would like to commit data once in a while after it has been retrieved and correctly processed and then fetch more. One option could b

Re: org.apache.spark.shuffle.FetchFailedException: Too large frame:

2018-05-03 Thread Ryan Blue
Yes, you can usually use a broadcast join to avoid skew problems. On Wed, May 2, 2018 at 8:57 PM, Pralabh Kumar wrote: > I am performing join operation , if I convert reduce side join to map side > (no shuffle will happen) and I assume in that case this error shouldn't > come. Let me know if th

Re: [Structured streaming, V2] commit on ContinuousReader

2018-05-03 Thread Joseph Torres
In the master branch, we currently call this method in ContinuousExecution.commit(). Note that the ContinuousReader API is experimental and undergoing active design work. We will definitely include some kind of functionality to back-commit data once it's been processed, but the handle we eventuall

Re: Identifying specific persisted DataFrames via getPersistentRDDs()

2018-05-03 Thread Reynold Xin
Why do you need the underlying RDDs? Can't you just unpersist the dataframes that you don't need? On Mon, Apr 30, 2018 at 8:17 PM Nicholas Chammas wrote: > This seems to be an underexposed part of the API. My use case is this: I > want to unpersist all DataFrames except a specific few. I want t