Re: A proposal for creating a Knowledge Sharing Hub for Apache Spark Community

2024-03-18 Thread Reynold Xin
One of the problem in the past when something like this was brought up was that the ASF couldn't have officially blessed venues beyond the already approved ones. So that's something to look into. Now of course you are welcome to run unofficial things unblessed as long as they follow trademark

Re: Stickers and Swag

2022-06-14 Thread Reynold Xin
Nice! Going to order a few items myself ... On Tue, Jun 14, 2022 at 7:54 PM, Gengliang Wang < ltn...@gmail.com > wrote: > > FYI now you can find the shopping information on https:/ / spark. apache. org/ > community ( https://spark.apache.org/community ) as well :) > > > > Gengliang > > >

[ANNOUNCE] Apache Spark 3.0.0

2020-06-18 Thread Reynold Xin
Hi all, Apache Spark 3.0.0 is the first release of the 3.x line. It builds on many of the innovations from Spark 2.x, bringing new ideas as well as continuing long-term projects that have been in development. This release resolves more than 3400 tickets. We'd like to thank our contributors

Re: results of taken(3) not appearing in console window

2020-03-26 Thread Reynold Xin
bcc dev, +user You need to print out the result. Take itself doesn't print. You only got the results printed to the console because the Scala REPL automatically prints the returned value from take. On Thu, Mar 26, 2020 at 12:15 PM, Zahid Rahman < zahidr1...@gmail.com > wrote: > > I am

Re: FYI: The evolution on `CHAR` type behavior

2020-03-16 Thread Reynold Xin
so deviate away from the standard on this specific behavior. On Mon, Mar 16, 2020 at 5:29 PM, Reynold Xin < r...@databricks.com > wrote: > > I looked up our usage logs (sorry I can't share this publicly) and trim > has at least four orders of magnitude higher usage than char. >

Re: FYI: The evolution on `CHAR` type behavior

2020-03-16 Thread Reynold Xin
joon.h...@gmail.com ) > wrote: >>> >>> Hi, Reynold. >>> (And +Michael Armbrust) >>> >>> >>> If you think so, do you think it's okay that we change the return value >>> silently? Then, I'm wondering why we reverted `TRIM`

Re: FYI: The evolution on `CHAR` type behavior

2020-03-16 Thread Reynold Xin
>> 100% agree with Reynold. >> >> >> >> >> Regards, >> Gourav Sengupta >> >> >> On Mon, Mar 16, 2020 at 3:31 AM Reynold Xin < rxin@ databricks. com ( >> r...@databricks.com ) > wrote: >> >> >>> Are

Re: FYI: The evolution on `CHAR` type behavior

2020-03-15 Thread Reynold Xin
the proposed alternative to reduce the potential issue. > > > Please give us your opinion since it's still PR. > > > Bests, > Dongjoon. > > On Sat, Mar 14, 2020 at 17:54 Reynold Xin < rxin@ databricks. com ( > r...@databricks.com ) > wrote: > > >>

Re: FYI: The evolution on `CHAR` type behavior

2020-03-14 Thread Reynold Xin
I don’t understand this change. Wouldn’t this “ban” confuse the hell out of both new and old users? For old users, their old code that was working for char(3) would now stop working. For new users, depending on whether the underlying metastore char(3) is either supported but different from ansi

Re: Collections passed from driver to executors

2019-09-23 Thread Reynold Xin
t; We are still at 2.2. > > On Tue, 24 Sep, 2019, 9:17 AM Reynold Xin, < rxin@ databricks. com ( > r...@databricks.com ) > wrote: > > >> A while ago we changed it so the task gets broadcasted too, so I think the >> two are fairly similar. >> >> >>

Re: Collections passed from driver to executors

2019-09-23 Thread Reynold Xin
A while ago we changed it so the task gets broadcasted too, so I think the two are fairly similar. On Mon, Sep 23, 2019 at 8:17 PM, Dhrubajyoti Hati < dhruba.w...@gmail.com > wrote: > > I was wondering if anyone could help with this question. > > On Fri, 20 Sep, 2019, 11:52 AM Dhrubajyoti

Re: Help: What's the biggest length of SQL that's supported in SparkSQL?

2019-07-12 Thread Reynold Xin
's. Any samples to share :)  > > > Regards, > Gourav > > On Thu, Jul 11, 2019 at 5:03 PM Reynold Xin < rxin@ databricks. com ( > r...@databricks.com ) > wrote: > > >> There is no explicit limit but a JVM string cannot be bigger than 2G. It >> will also

Re: Problems running TPC-H on Raspberry Pi Cluster

2019-07-11 Thread Reynold Xin
I don't think Spark is meant to run with 1GB of memory on the entire system. The JVM loads almost 200MB of bytecode, and each page during query processing takes a min of 64MB. Maybe on the 4GB model of raspberry pi 4. On Wed, Jul 10, 2019 at 7:57 AM, agg212 < alexander_galaka...@brown.edu >

Re: Help: What's the biggest length of SQL that's supported in SparkSQL?

2019-07-11 Thread Reynold Xin
There is no explicit limit but a JVM string cannot be bigger than 2G. It will also at some point run out of memory with too big of a query plan tree or become incredibly slow due to query planning complexity. I've seen queries that are tens of MBs in size. On Thu, Jul 11, 2019 at 5:01 AM, 李书明

Re: Exposing JIRA issue types at GitHub PRs

2019-06-13 Thread Reynold Xin
Seems like a good idea. Can we test this with a component first? On Thu, Jun 13, 2019 at 6:17 AM Dongjoon Hyun wrote: > Hi, All. > > Since we use both Apache JIRA and GitHub actively for Apache Spark > contributions, we have lots of JIRAs and PRs consequently. One specific > thing I've been

Re: Should python-2 be supported in Spark 3.0?

2019-05-30 Thread Reynold Xin
+1 on Xiangrui’s plan. On Thu, May 30, 2019 at 7:55 AM shane knapp wrote: > I don't have a good sense of the overhead of continuing to support >> Python 2; is it large enough to consider dropping it in Spark 3.0? >> >> from the build/test side, it will actually be pretty easy to continue >

Re: Koalas show data in IDE or pyspark

2019-05-14 Thread Reynold Xin
This has been fixed and was included in the release 0.3 last week. We will be making another release (0.4) in the next 24 hours to include more features also. On Tue, Apr 30, 2019 at 12:42 AM, Manu Zhang < owenzhang1...@gmail.com > wrote: > > Hi, > > > It seems koalas.DataFrame can't be

Re: [HELP WANTED] Apache Zipkin (incubating) needs Spark gurus

2019-03-21 Thread Reynold Xin
Are there specific questions you have? Might be easier to post them here also. On Wed, Mar 20, 2019 at 5:16 PM Andriy Redko wrote: > Hello Dear Spark Community! > > The hyper-popularity of the Apache Spark made it a de-facto choice for many > projects which need some sort of data processing

Re: [PySpark] Revisiting PySpark type annotations

2019-01-25 Thread Reynold Xin
If we can make the annotation compatible with Python 2, why don’t we add type annotation to make life easier for users of Python 3 (with type)? On Fri, Jan 25, 2019 at 7:53 AM Maciej Szymkiewicz wrote: > > Hello everyone, > > I'd like to revisit the topic of adding PySpark type annotations in

Re: [ANNOUNCE] Announcing Apache Spark 2.4.0

2018-11-08 Thread Reynold Xin
Do you have a cached copy? I see it here http://spark.apache.org/downloads.html On Thu, Nov 8, 2018 at 4:12 PM Li Gao wrote: > this is wonderful ! > I noticed the official spark download site does not have 2.4 download > links yet. > > On Thu, Nov 8, 2018, 4:11 PM Swapnil Shinde wrote: > >>

Re: Back to SQL

2018-10-03 Thread Reynold Xin
No we used to have that (for views) but it wasn’t working well enough so we removed it. On Wed, Oct 3, 2018 at 6:41 PM Olivier Girardot < o.girar...@lateral-thoughts.com> wrote: > Hi everyone, > Is there any known way to go from a Spark SQL Logical Plan (optimised ?) > Back to a SQL query ? > >

Re: Should python-2 be supported in Spark 3.0?

2018-09-17 Thread Reynold Xin
i'd like to second that. if we want to communicate timeline, we can add to the release notes saying py2 will be deprecated in 3.0, and removed in a 3.x release. -- excuse the brevity and lower case due to wrist injury On Mon, Sep 17, 2018 at 4:24 PM Matei Zaharia wrote: > That’s a good point

Re: Fw:multiple group by action

2018-08-24 Thread Reynold Xin
Use rollout and cube. On Fri, Aug 24, 2018 at 7:55 PM 崔苗 wrote: > > > > > > > Forwarding messages > From: "崔苗" > Date: 2018-08-25 10:54:31 > To: d...@spark.apache.org > Subject: multiple group by action > > Hi, > we have some user data with >

Re: Anyone knows how to build and spark on jdk9?

2017-10-26 Thread Reynold Xin
It probably depends on the Scala version we use in Spark supporting Java 9 first. On Thu, Oct 26, 2017 at 7:22 PM Zhang, Liyun wrote: > Hi all: > > 1. I want to build spark on jdk9 and test it with Hadoop on jdk9 > env. I search for jiras related to JDK9. I only

Re: Dataset API Question

2017-10-25 Thread Reynold Xin
It is a bit more than syntactic sugar, but not much more: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala#L533 BTW this is basically writing all the data out, and then create a new Dataset to load them in. On Wed, Oct 25, 2017 at 6:51 AM,

Re: SQL specific documentation for recent Spark releases

2017-08-11 Thread Reynold Xin
This PR should help you in the next release: https://github.com/apache/spark/pull/18702 On Thu, Aug 10, 2017 at 7:46 PM, Stephen Boesch wrote: > > The correct link is https://docs.databricks.com/ > spark/latest/spark-sql/index.html . > > This link does have the core syntax

Re: Question on Spark code

2017-07-23 Thread Reynold Xin
, it will return the same type with the level that you called it. >> >> On Sun, Jul 23, 2017 at 8:20 PM Reynold Xin <r...@databricks.com> wrote: >> >>> It means the same object ("this") is returned. >>> >>> On Sun,

Re: Question on Spark code

2017-07-23 Thread Reynold Xin
It means the same object ("this") is returned. On Sun, Jul 23, 2017 at 8:16 PM, tao zhan wrote: > Hello, > > I am new to scala and spark. > What does the "this.type" in set function for? > > > ​ > https://github.com/apache/spark/blob/481f0792944d9a77f0fe8b5e2596da >

Re: the dependence length of RDD, can its size be greater than 1 pleaae?

2017-06-15 Thread Reynold Xin
A join? On Thu, Jun 15, 2017 at 1:11 AM 萝卜丝炒饭 <1427357...@qq.com> wrote: > Hi all, > > The RDD code keeps a member as below: > dependencies_ : seq[Dependency[_]] > > It is a seq, that means it can keep more than one dependency. > > I have an issue about this. > Is it possible that its size is

Re: is dataframe thread safe?

2017-02-13 Thread Reynold Xin
Yes your use case should be fine. Multiple threads can transform the same data frame in parallel since they create different data frames. On Sun, Feb 12, 2017 at 9:07 AM Mendelson, Assaf wrote: > Hi, > > I was wondering if dataframe is considered thread safe. I know

Re: Output Side Effects for different chain of operations

2016-12-15 Thread Reynold Xin
You can just write some files out directly (and idempotently) in your map/mapPartitions functions. It is just a function that you can run arbitrary code after all. On Thu, Dec 15, 2016 at 11:33 AM, Chawla,Sumit wrote: > Any suggestions on this one? > > Regards > Sumit

Re: Can't read tables written in Spark 2.1 in Spark 2.0 (and earlier)

2016-11-30 Thread Reynold Xin
This should fix it: https://github.com/apache/spark/pull/16080 On Wed, Nov 30, 2016 at 10:55 AM, Timur Shenkao wrote: > Hello, > > Yes, I used hiveContext, sqlContext, sparkSession from Java, Scala, > Python. > Via spark-shell, spark-submit, IDE (PyCharm, Intellij IDEA). >

Re: Bit-wise AND operation between integers

2016-11-28 Thread Reynold Xin
Bcc dev@ and add user@ The dev list is not meant for users to ask questions on how to use Spark. For that you should use StackOverflow or the user@ list. scala> sql("select 1 & 2").show() +---+ |(1 & 2)| +---+ | 0| +---+ scala> sql("select 1 & 3").show() +---+ |(1 & 3)|

Re: Third party library

2016-11-26 Thread Reynold Xin
o-C-library. > Is I am missing something ? If possible, can you point me to existing > implementation which i can refer to. > > Thanks again. > > ~ > > On Fri, Nov 25, 2016 at 3:32 PM, Reynold Xin <r...@databricks.com> wrote: > >> bcc dev@ and add us

Re: Third party library

2016-11-25 Thread Reynold Xin
bcc dev@ and add user@ This is more a user@ list question rather than a dev@ list question. You can do something like this: object MySimpleApp { def loadResources(): Unit = // define some idempotent way to load resources, e.g. with a flag or lazy val def main() = { ...

Re: Re: Re: Multiple streaming aggregations in structured streaming

2016-11-22 Thread Reynold Xin
It's just the "approx_count_distinct" aggregate function. On Tue, Nov 22, 2016 at 6:51 PM, Xinyu Zhang <wsz...@163.com> wrote: > Could you please tell me how to use the approximate count distinct? Is > there any docs? > > Thanks > > > At 2016-11-21 15:56:2

Re: Re: Multiple streaming aggregations in structured streaming

2016-11-20 Thread Reynold Xin
Can you use the approximate count distinct? On Sun, Nov 20, 2016 at 11:51 PM, Xinyu Zhang wrote: > > MapWithState is also very useful. > I want to calculate UV in real time, but "distinct count" and "multiple > streaming aggregations" are not supported. > Is there any method to

Re: How do I convert json_encoded_blob_column into a data frame? (This may be a feature request)

2016-11-17 Thread Reynold Xin
Adding a new data type is an enormous undertaking and very invasive. I don't think it is worth it in this case given there are clear, simple workarounds. On Thu, Nov 17, 2016 at 12:24 PM, kant kodali wrote: > Can we have a JSONType for Spark SQL? > > On Wed, Nov 16, 2016 at

[ANNOUNCE] Apache Spark 2.0.2

2016-11-14 Thread Reynold Xin
We are happy to announce the availability of Spark 2.0.2! Apache Spark 2.0.2 is a maintenance release containing 90 bug fixes along with Kafka 0.10 support and runtime metrics for Structured Streaming. This release is based on the branch-2.0 maintenance branch of Spark. We strongly recommend all

[ANNOUNCE] Announcing Apache Spark 1.6.3

2016-11-07 Thread Reynold Xin
We are happy to announce the availability of Spark 1.6.3! This maintenance release includes fixes across several areas of Spark and encourage users on the 1.6.x line to upgrade to 1.6.3. Head to the project's download page to download the new version: http://spark.apache.org/downloads.html

Mark DataFrame/Dataset APIs stable

2016-10-12 Thread Reynold Xin
I took a look at all the public APIs we expose in o.a.spark.sql tonight, and realized we still have a large number of APIs that are marked experimental. Most of these haven't really changed, except in 2.0 we merged DataFrame and Dataset. I think it's long overdue to mark them stable. I'm tracking

Re: [SPARK-2.0][SQL] UDF containing non-serializable object does not work as expected

2016-08-08 Thread Reynold Xin
> > The workaround I can imagine is just to cache and materialize `df` by > `df.cache.count()`, and then call `df.filter(...).show()`. > It should work, just a little bit tedious. > > > > On Mon, Aug 8, 2016 at 10:00 PM, Reynold Xin <r...@databricks.com> wrote: >

Re: [SPARK-2.0][SQL] UDF containing non-serializable object does not work as expected

2016-08-08 Thread Reynold Xin
That is unfortunately the way how Scala compiler captures (and defines) closures. Nothing is really final in the JVM. You can always use reflection or unsafe to modify the value of fields. On Mon, Aug 8, 2016 at 8:16 PM, Simon Scott wrote: > But does the “notSer”

Re: RDD vs Dataset performance

2016-07-28 Thread Reynold Xin
The performance difference is coming from the need to serialize and deserialize data to AnnotationText. The extra stage is probably very quick and shouldn't impact much. If you try cache the RDD using serialized mode, it would slow down a lot too. On Thu, Jul 28, 2016 at 9:52 AM, Darin McBeath

[ANNOUNCE] Announcing Apache Spark 2.0.0

2016-07-27 Thread Reynold Xin
Hi all, Apache Spark 2.0.0 is the first release of Spark 2.x line. It includes 2500+ patches from 300+ contributors. To download Spark 2.0, head over to the download page: http://spark.apache.org/downloads.html To view the release notes: http://spark.apache.org/releases/spark-release-2-0-0.html

Re: where I can find spark-streaming-kafka for spark2.0

2016-07-25 Thread Reynold Xin
The presentation at Spark Summit SF was probably referring to Structured Streaming. The existing Spark Streaming (dstream) in Spark 2.0 has the same production stability level as Spark 1.6. There is also Kafka 0.10 support in dstream. On July 25, 2016 at 10:26:49 AM, Andy Davidson (

Re: transtition SQLContext to SparkSession

2016-07-19 Thread Reynold Xin
Yes. But in order to access methods available only in HiveContext a user cast is required. On Tuesday, July 19, 2016, Maciej Bryński <mac...@brynski.pl> wrote: > @Reynold Xin, > How this will work with Hive Support ? > SparkSession.sqlContext return HiveContext ? > > 2016

Re: transtition SQLContext to SparkSession

2016-07-18 Thread Reynold Xin
Good idea. https://github.com/apache/spark/pull/14252 On Mon, Jul 18, 2016 at 12:16 PM, Michael Armbrust wrote: > + dev, reynold > > Yeah, thats a good point. I wonder if SparkSession.sqlContext should be > public/deprecated? > > On Mon, Jul 18, 2016 at 8:37 AM,

Re: Spark Website

2016-07-13 Thread Reynold Xin
Thanks for reporting. This is due to https://issues.apache.org/jira/servicedesk/agent/INFRA/issue/INFRA-12055 On Wed, Jul 13, 2016 at 11:52 AM, Pradeep Gollakota wrote: > Worked for me if I go to https://spark.apache.org/site/ but not > https://spark.apache.org > > On

Re: ml and mllib persistence

2016-07-12 Thread Reynold Xin
Also Java serialization isn't great for cross platform compatibility. On Tuesday, July 12, 2016, aka.fe2s wrote: > Okay, I think I found an answer on my question. Some models (for instance > org.apache.spark.mllib.recommendation.MatrixFactorizationModel) hold RDDs, > so just

Re: Logical Plan

2016-06-30 Thread Reynold Xin
Which version are you using here? If the underlying files change, technically we should go through optimization again. Perhaps the real "fix" is to figure out why is logical plan creation so slow for 700 columns. On Thu, Jun 30, 2016 at 1:58 PM, Darshan Singh wrote: >

[ANNOUNCE] Announcing Spark 1.6.2

2016-06-27 Thread Reynold Xin
We are happy to announce the availability of Spark 1.6.2! This maintenance release includes fixes across several areas of Spark. You can find the list of changes here: https://s.apache.org/spark-1.6.2 And download the release here: http://spark.apache.org/downloads.html

Re: Thanks For a Job Well Done !!!

2016-06-18 Thread Reynold Xin
Thanks for the kind words, Krishna! Please keep the feedback coming. On Saturday, June 18, 2016, Krishna Sankar wrote: > Hi all, >Just wanted to thank all for the dataset API - most of the times we see > only bugs in these lists ;o). > >- Putting some context, this

Re: Spark 2.0 Release Date

2016-06-07 Thread Reynold Xin
It'd be great to cut an RC as soon as possible. Looking at the blocker/critical issue list, majority of them are API audits. I think people will get back to those once Spark Summit is over, and then we should see some good progress towards an RC. On Tue, Jun 7, 2016 at 6:20 AM, Jacek Laskowski

Re: JDBC Dialect for saving DataFrame into Vertica Table

2016-05-26 Thread Reynold Xin
It's probably a good idea to have the vertica dialect too, since it doesn't seem like it'd be too difficult to maintain. It is not going to be as performant as the native Vertica data source, but is going to be much lighter weight. On Thu, May 26, 2016 at 3:09 PM, Mohammed Guller

Re: feedback on dataset api explode

2016-05-25 Thread Reynold Xin
ee, since they can be easily replaced by .flatMap (to do explosion) and > .select (to rename output columns) > > Cheng > > > On 5/25/16 12:30 PM, Reynold Xin wrote: > > Based on this discussion I'm thinking we should deprecate the two explode > functions. > > On We

Re: Pros and Cons

2016-05-25 Thread Reynold Xin
On Wed, May 25, 2016 at 9:52 AM, Jörn Franke wrote: > Spark is more for machine learning working iteravely over the whole same > dataset in memory. Additionally it has streaming and graph processing > capabilities that can be used together. > Hi Jörn, The first part is

Re: feedback on dataset api explode

2016-05-25 Thread Reynold Xin
Based on this discussion I'm thinking we should deprecate the two explode functions. On Wednesday, May 25, 2016, Koert Kuipers wrote: > wenchen, > that definition of explode seems identical to flatMap, so you dont need it > either? > > michael, > i didn't know about the

Re: Spark 2.0 - SQL Subqueries.

2016-05-21 Thread Reynold Xin
https://issues.apache.org/jira/browse/SPARK-15078 was just a bunch of test harness and added no new functionality. To reduce confusion, I just backported it into branch-2.0 so SPARK-15078 is now in 2.0 too. Can you paste a query you were testing? On Sat, May 21, 2016 at 10:49 AM, Kamalesh Nair

Re: Do transformation functions on RDD invoke a Job [sc.runJob]?

2016-04-24 Thread Reynold Xin
Usually no - but sortByKey does because it needs the range boundary to be built in order to have the RDD. It is a long standing problem that's unfortunately very difficult to solve without breaking the RDD API. In DataFrame/Dataset we don't have this issue though. On Sun, Apr 24, 2016 at 10:54

Re: How Spark handles dead machines during a job.

2016-04-09 Thread Reynold Xin
The driver has the data and wouldn't need to rerun. On Friday, April 8, 2016, Sung Hwan Chung wrote: > Hello, > > Say, that I'm doing a simple rdd.map followed by collect. Say, also, that > one of the executors finish all of its tasks, but there are still other > executors

Re: Executor shutdown hooks?

2016-04-06 Thread Reynold Xin
On Wed, Apr 6, 2016 at 4:39 PM, Sung Hwan Chung wrote: > My option so far seems to be using JVM's shutdown hook, but I was > wondering if Spark itself had an API for tasks. > Spark would be using that under the hood anyway, so you might as well just use the jvm

Re: Switch RDD-based MLlib APIs to maintenance mode in Spark 2.0

2016-04-05 Thread Reynold Xin
+1 This is a no brainer IMO. On Tue, Apr 5, 2016 at 7:32 PM, Joseph Bradley wrote: > +1 By the way, the JIRA for tracking (Scala) API parity is: > https://issues.apache.org/jira/browse/SPARK-4591 > > On Tue, Apr 5, 2016 at 4:58 PM, Matei Zaharia

Re: Eliminating shuffle write and spill disk IO reads/writes in Spark

2016-04-01 Thread Reynold Xin
cal.dir as a buffer pool of > > others. > > > > Hence, the performance of Spark is gated by the performance of > > spark.local.dir, even on large memory systems. > > > > "Currently it is not possible to not write shuffle files to disk.” > > > > What c

Re: Eliminating shuffle write and spill disk IO reads/writes in Spark

2016-04-01 Thread Reynold Xin
ings but use spark.local.dir as a buffer pool of >> > others. >> > >> > Hence, the performance of Spark is gated by the performance of >> > spark.local.dir, even on large memory systems. >> > >> > "Currently it is not possible to not write shu

Re: Eliminating shuffle write and spill disk IO reads/writes in Spark

2016-04-01 Thread Reynold Xin
spark.shuffle.spill actually has nothing to do with whether we write shuffle files to disk. Currently it is not possible to not write shuffle files to disk, and typically it is not a problem because the network fetch throughput is lower than what disks can sustain. In most cases, especially with

Re: df.dtypes -> pyspark.sql.types

2016-03-20 Thread Reynold Xin
We probably should have the alias. Is this still a problem on master branch? On Wed, Mar 16, 2016 at 9:40 AM, Ruslan Dautkhanov wrote: > Running following: > > #fix schema for gaid which should not be Double >> from pyspark.sql.types import * >> customSchema = StructType()

Re: [discuss] making SparkEnv private in Spark 2.0

2016-03-19 Thread Reynold Xin
On Wed, Mar 16, 2016 at 3:29 PM, Mridul Muralidharan wrote: > b) Shuffle manager (to get shuffle reader) > What's the use case for shuffle manager/reader? This seems like using super internal APIs in applications.

[discuss] making SparkEnv private in Spark 2.0

2016-03-19 Thread Reynold Xin
Any objections? Please articulate your use case. SparkEnv is a weird one because it was documented as "private" but not marked as so in class visibility. * NOTE: This is not intended for external use. This is exposed for Shark and may be made private * in a future release. I do see Hive

Re: Spark Scheduler creating Straggler Node

2016-03-08 Thread Reynold Xin
You just want to be able to replicate hot cached blocks right? On Tuesday, March 8, 2016, Prabhu Joseph wrote: > Hi All, > > When a Spark Job is running, and one of the Spark Executor on Node A > has some partitions cached. Later for some other stage, Scheduler

Re: Selecting column in dataframe created with incompatible schema causes AnalysisException

2016-03-02 Thread Reynold Xin
Thanks. Once you create the jira just reply to this email with the link. On Wednesday, March 2, 2016, Ewan Leith wrote: > Thanks, I'll create the JIRA for it. Happy to help contribute to a patch if > we can, not sure if my own scala skills will be up to it but

Re: Selecting column in dataframe created with incompatible schema causes AnalysisException

2016-03-02 Thread Reynold Xin
I don't think that exists right now, but it's definitely a good option to have. I myself have run into this issue a few times. Can you create a JIRA ticket so we can track it? Would be even better if you are interested in working on a patch! Thanks. On Wed, Mar 2, 2016 at 11:51 AM, Ewan Leith

Re: Selecting column in dataframe created with incompatible schema causes AnalysisException

2016-03-02 Thread Reynold Xin
Are you looking for "relaxed" mode that simply return nulls for fields that doesn't exist or have incompatible schema? On Wed, Mar 2, 2016 at 11:12 AM, Ewan Leith wrote: > Thanks Michael, it's not a great example really, as the data I'm working with > has some

Re: [Proposal] Enabling time series analysis on spark metrics

2016-03-01 Thread Reynold Xin
Is the suggestion just to use a different config (and maybe fallback to appid) in order to publish metrics? Seems reasonable. On Tue, Mar 1, 2016 at 8:17 AM, Karan Kumar wrote: > +dev mailing list > > Time series analysis on metrics becomes quite useful when running

Re: Is spark.driver.maxResultSize used correctly ?

2016-03-01 Thread Reynold Xin
data skew might be possible, but not the common case. I think we should > design for the common case, for the skew case, we may can set some > parameter of fraction to allow user to tune it. > > On Sat, Feb 27, 2016 at 4:51 PM, Reynold Xin <r...@databricks.com > <javascript:_e(%7B%

Re: Is spark.driver.maxResultSize used correctly ?

2016-02-27 Thread Reynold Xin
But sometimes you might have skew and almost all the result data are in one or a few tasks though. On Friday, February 26, 2016, Jeff Zhang wrote: > > My job get this exception very easily even when I set large value of > spark.driver.maxResultSize. After checking the spark

Re: DirectFileOutputCommiter

2016-02-26 Thread Reynold Xin
It could lose data in speculation mode, or if any job fails. On Fri, Feb 26, 2016 at 3:45 AM, Igor Berman wrote: > Takeshi, do you know the reason why they wanted to remove this commiter in > SPARK-10063? > the jira has no info inside > as far as I understand the direct

Spark Summit (San Francisco, June 6-8) call for presentation due in less than week

2016-02-24 Thread Reynold Xin
Just want to send a reminder in case people don't know about it. If you are working on (or with, using) Spark, consider submitting your work to Spark Summit, coming up in June in San Francisco. https://spark-summit.org/2016/call-for-presentations/ Cheers.

[discuss] dropping Hadoop 2.2 and 2.3 support in Spark 2.0?

2016-01-13 Thread Reynold Xin
We've dropped Hadoop 1.x support in Spark 2.0. There is also a proposal to drop Hadoop 2.2 and 2.3, i.e. the minimal Hadoop version we support would be Hadoop 2.4. The main advantage is then we'd be able to focus our Jenkins resources (and the associated maintenance of Jenkins) to create builds

Re: XML column not supported in Database

2016-01-11 Thread Reynold Xin
Can you file a JIRA ticket? Thanks. The URL is issues.apache.org/jira/browse/SPARK On Mon, Jan 11, 2016 at 1:44 AM, Gaini Rajeshwar < raja.rajeshwar2...@gmail.com> wrote: > Hi All, > > I am using PostgreSQL database. I am using the following jdbc call to > access a customer table (*customer_id

[discuss] dropping Python 2.6 support

2016-01-04 Thread Reynold Xin
Does anybody here care about us dropping support for Python 2.6 in Spark 2.0? Python 2.6 is ancient, and is pretty slow in many aspects (e.g. json parsing) when compared with Python 2.7. Some libraries that Spark depend on stopped supporting 2.6. We can still convince the library maintainers to

Re: Please add us to the Powered by Spark page

2015-11-24 Thread Reynold Xin
I just updated the page to say "email dev" instead of "email user". On Tue, Nov 24, 2015 at 1:16 AM, Sean Owen wrote: > Not sure who generally handles that, but I just made the edit. > > On Mon, Nov 23, 2015 at 6:26 PM, Sujit Pal wrote: > > Sorry to

Re: orc read issue n spark

2015-11-18 Thread Reynold Xin
What do you mean by starts delay scheduling? Are you saying it is no longer doing local reads? If that's the case you can increase the spark.locality.read timeout. On Wednesday, November 18, 2015, Renu Yadav wrote: > Hi , > I am using spark 1.4.1 and saving orc file using >

Re: Hive on Spark Vs Spark SQL

2015-11-15 Thread Reynold Xin
It's a completely different path. On Sun, Nov 15, 2015 at 10:37 PM, kiran lonikar wrote: > I would like to know if Hive on Spark uses or shares the execution code > with Spark SQL or DataFrames? > > More specifically, does Hive on Spark benefit from the changes made to >

Re: Hive on Spark Vs Spark SQL

2015-11-15 Thread Reynold Xin
No it does not -- although it'd benefit from some of the work to make shuffle more robust. On Sun, Nov 15, 2015 at 10:45 PM, kiran lonikar <loni...@gmail.com> wrote: > So does not benefit from Project Tungsten right? > > > On Mon, Nov 16, 2015 at 12:07 PM, Reynold Xin &l

[ANNOUNCE] Announcing Spark 1.5.2

2015-11-10 Thread Reynold Xin
Hi All, Spark 1.5.2 is a maintenance release containing stability fixes. This release is based on the branch-1.5 maintenance branch of Spark. We *strongly recommend* all 1.5.x users to upgrade to this release. The full list of bug fixes is here: http://s.apache.org/spark-1.5.2

Re: Looking for the method executors uses to write to HDFS

2015-11-06 Thread Reynold Xin
Are you looking for this? https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala#L69 On Wed, Nov 4, 2015 at 5:11 AM, Tóth Zoltán wrote: > Hi, > > I'd like to write a parquet file from the

Re: Codegen In Shuffle

2015-11-04 Thread Reynold Xin
GenerateUnsafeProjection -- projects any internal row data structure directly into bytes (UnsafeRow). On Wed, Nov 4, 2015 at 12:21 AM, 牛兆捷 wrote: > Dear all: > > Tungsten project has mentioned that they are applying code generation is > to speed up the conversion of data

Please reply if you use Mesos fine grained mode

2015-11-03 Thread Reynold Xin
If you are using Spark with Mesos fine grained mode, can you please respond to this email explaining why you use it over the coarse grained mode? Thanks.

Re: Please reply if you use Mesos fine grained mode

2015-11-03 Thread Reynold Xin
in turn kill the entire executor, causing entire > stages to be retried. In fine-grained mode, only the task fails and > subsequently gets retried without taking out an entire stage or worse. > > On Tue, Nov 3, 2015 at 3:54 PM, Reynold Xin <r...@databricks.com> wrote: > >>

Re: If you use Spark 1.5 and disabled Tungsten mode ...

2015-11-01 Thread Reynold Xin
$sql$execution$TungstenSort$$preparePartition$1(sort.scala:131) >>> at >>> org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.scala:169) >>> at >>> org.apache.spark.sql.execution.TungstenSort$$anonfun$doExecute$3.apply(sort.s

Re: [SQL] Memory leak with spark streaming and spark sql in spark 1.5.1

2015-10-14 Thread Reynold Xin
+dev list On Wed, Oct 14, 2015 at 1:07 AM, Terry Hoo wrote: > All, > > Does anyone meet memory leak issue with spark streaming and spark sql in > spark 1.5.1? I can see the memory is increasing all the time when running > this simple sample: > > val sc = new

If you use Spark 1.5 and disabled Tungsten mode ...

2015-10-14 Thread Reynold Xin
Can you reply to this email and provide us with reasons why you disable it? Thanks.

[ANNOUNCE] Announcing Spark 1.5.1

2015-10-01 Thread Reynold Xin
Hi All, Spark 1.5.1 is a maintenance release containing stability fixes. This release is based on the branch-1.5 maintenance branch of Spark. We *strongly recommend* all 1.5.0 users to upgrade to this release. The full list of bug fixes is here: http://s.apache.org/spark-1.5.1

Re: Null Value in DecimalType column of DataFrame

2015-09-21 Thread Reynold Xin
+dev list Hi Dirceu, The answer to whether throwing an exception is better or null is better depends on your use case. If you are debugging and want to find bugs with your program, you might prefer throwing an exception. However, if you are running on a large real-world dataset (i.e. data is

Re: in joins, does one side stream?

2015-09-20 Thread Reynold Xin
n DataFrame >> but not in RDD? >> >> they dont seem specific to structured data analysis to me. >> >> On Sun, Sep 20, 2015 at 2:41 AM, Rishitesh Mishra < >> rishi80.mis...@gmail.com> wrote: >> >>> Got it..thnx Reynold.. >>&g

Re: in joins, does one side stream?

2015-09-19 Thread Reynold Xin
eynold, > Can you please elaborate on this. I thought RDD also opens only an > iterator. Does it get materialized for joins? > > Rishi > > On Saturday, September 19, 2015, Reynold Xin <r...@databricks.com> wrote: > >> Yes for RDD -- both are materializ

Re: in joins, does one side stream?

2015-09-18 Thread Reynold Xin
Yes for RDD -- both are materialized. No for DataFrame/SQL - one side streams. On Thu, Sep 17, 2015 at 11:21 AM, Koert Kuipers wrote: > in scalding we join with the smaller side on the left, since the smaller > side will get buffered while the bigger side streams through the

Re: How to avoid shuffle errors for a large join ?

2015-09-16 Thread Reynold Xin
Only SQL and DataFrame for now. We are thinking about how to apply that to a more general distributed collection based API, but it's not in 1.5. On Sat, Sep 5, 2015 at 11:56 AM, Gurvinder Singh <gurvinder.si...@uninett.no > wrote: > On 09/05/2015 11:22 AM, Reynold Xin wrote: > &g

Re: Perf impact of BlockManager byte[] copies

2015-09-10 Thread Reynold Xin
This is one problem I'd like to address soon - providing a binary block management interface for shuffle (and maybe other things) that avoids serialization/copying. On Fri, Feb 27, 2015 at 3:39 PM, Paul Wais wrote: > Dear List, > > I'm investigating some problems related to

  1   2   >