afe methods
Another option is to ask the PMC to vote for blocking Spark 3.0 if the
"return null behavior" is not fixed. But I don't think it's likely to
happen.
On Tue, Aug 6, 2019 at 12:34 AM Ryan Blue wrote:
> Wenchen, I don’t think we agree on what “strict mode” would mean. Mar
work to educate users to use the safe version of the
functions if they rely on the return null behavior.
* turn on the strict mode by default.
Hopefully we can finish it soon, in Spark 3.x.
Thanks,
Wenchen
On Sat, Aug 3, 2019 at 7:07 AM Matt Cheah wrote:
> *I agree that having both modes and le
nal
> effect of the null decision would be different depending on the insertion
> target if the target has different behaviors for null.
>
> On Mon, Jul 29, 2019 at 5:26 AM Wenchen Fan wrote:
>
>> > I'm a big -1 on null values for invalid casts.
>>
>> This is why
behavior
>>> anyway.
>>> Eventually, most sources are supposed to be migrated to DataSourceV2 V2.
>>> I think we can discuss and make a decision now.
>>>
>>> > Fixing the silent corruption by adding a runtime exception is not a
>>> good opti
You can try EXPLAIN COST query and see if it works for you.
On Mon, Jul 29, 2019 at 5:34 PM Rubén Berenguel
wrote:
> I think there is no way of doing that (at least don't remember one right
> now). The closer I remember now, is you can run the SQL "ANALYZE TABLE
> table_name COMPUTE STATISTIC"
mals and fail simple insert
> statements. We already came up with two alternatives to fix that problem in
> the DSv2 sync and I think it is a better idea to go with one of those
> instead of “fixing” Spark in a way that will corrupt data or cause runtime
> failures.
>
> On T
than before and should
prevent a lot of user mistakes. It's also a reasonable choice to me to not
throw exceptions at runtime by default, as it's usually bad for
long-running jobs.
Thanks,
Wenchen
On Thu, Jul 25, 2019 at 11:37 PM Gengliang Wang <
gengliang.w...@databricks.com> wrote:
> Hi
Hi Ryan,
Thanks for summarizing and sending out the meeting notes! Unfortunately, I
missed the last sync, but the topics are really interesting, especially the
stats integration.
The ideal solution I can think of is to refactor the optimizer/planner and
move all the stats-based optimization to
+1 as well
On Tue, Jul 2, 2019 at 12:13 PM Dongjoon Hyun
wrote:
> Thank you so much for the replies, Reynold, Sean, Takeshi, Hyukjin!
>
> Bests,
> Dongjoon.
>
> On Mon, Jul 1, 2019 at 6:00 PM Hyukjin Kwon wrote:
>
>> +1
>>
>> 2019년 7월 2일 (화) 오전 9:39, Takeshi Yamamuro 님이 작성:
>>
>>> I'm also
You can call `SparkContext#addSparkListener` with a listener that
implements `onApplicationEnd`.
On Tue, May 14, 2019 at 1:51 AM Nasrulla Khan Haris
wrote:
> HI All,
>
>
>
> Is there a API for sparkContext where we can add our custom code before
> stopping sparkcontext ?
>
> Appreciate your
When you cache a dataframe, you actually cache a logical plan. That's why
re-creating the dataframe doesn't work: Spark finds out the logical plan is
cached and picks the cached data.
You need to uncache the dataframe, or go back to the SQL way:
df.createTempView("abc")
spark.table("abc").cache()
RDD is kind of a pointer to the actual data. Unless it's cached, we don't
need to clean up the RDD.
On Tue, May 21, 2019 at 1:48 PM Nasrulla Khan Haris
wrote:
> HI Spark developers,
>
>
>
> Can someone point out the code where RDD objects go out of scope ?. I
> found the contextcleaner
>
+1.
The Scala version problem has been resolved, which is the main motivation
of 2.4.3.
On Mon, May 6, 2019 at 12:38 AM Felix Cheung
wrote:
> I ran basic tests on R, r-hub etc. LGTM.
>
> +1 (limited - I didn’t get to run other usual tests)
>
> --
> *From:* Sean Owen
-port and re-roll the
> RC. That said I think we did / had to already drop the ability to build <=
> 2.3 from the master release script already.
>
> On Sun, Apr 28, 2019 at 9:25 PM Wenchen Fan wrote:
>
>> > ... by using the release script of Spark 2.4 branch
>>
&
ml)
>
> On Wed, Apr 24, 2019 at 6:24 PM Wenchen Fan wrote:
>
>> How did you read/write the timestamp value from/to ORC file?
>>
>> On Wed, Apr 24, 2019 at 6:30 PM Shubham Chaurasia <
>> shubh.chaura...@gmail.com> wrote:
>>
>>> Hi Al
How did you read/write the timestamp value from/to ORC file?
On Wed, Apr 24, 2019 at 6:30 PM Shubham Chaurasia
wrote:
> Hi All,
>
> Consider the following(spark v2.4.0):
>
> Basically I change values of `spark.sql.session.timeZone` and perform an
> orc write. Here are 3 samples:-
>
> 1)
>
this one for sure as it's easy to
> overlook with all the pages being updated per release.
>
> On Thu, Apr 18, 2019 at 9:51 PM Wenchen Fan wrote:
> >
> > Please vote on releasing the following candidate as Apache Spark version
> 2.4.2.
> >
> > The vote is open until
I've cut RC1. If people think we must upgrade Jackson in 2.4, I can cut RC2
shortly.
Thanks,
Wenchen
On Fri, Apr 19, 2019 at 3:32 AM Felix Cheung
wrote:
> Re shading - same argument I’ve made earlier today in a PR...
>
> (Context- in many cases Spark has light or indirect dep
Please vote on releasing the following candidate as Apache Spark version
2.4.2.
The vote is open until April 23 PST and passes if a majority +1 PMC votes
are cast, with
a minimum of 3 +1 votes.
[ ] +1 Release this package as Apache Spark 2.4.2
[ ] -1 Do not release this package because ...
To
I volunteer to be the release manager for 2.4.2, as I was also going to
propose 2.4.2 because of the reverting of SPARK-25250. Is there any other
ongoing bug fixes we want to include in 2.4.2? If no I'd like to start the
release process today (CST).
Thanks,
Wenchen
On Thu, Apr 18, 2019 at 3:44
Like `RDD.map`, you can throw whatever exceptions and they will be
propagated to the driver side and fail the Spark job.
On Mon, Apr 8, 2019 at 3:10 PM Andrew Melo wrote:
> Hello,
>
> I'm developing a (java) DataSourceV2 to read a columnar fileformat
> popular in a number of physical sciences
+1, all the known blockers are resolved. Thanks for driving this!
On Wed, Mar 27, 2019 at 11:31 AM DB Tsai wrote:
> Please vote on releasing the following candidate as Apache Spark version
> 2.4.1.
>
> The vote is open until March 30 PST and passes if a majority +1 PMC votes
> are cast, with
>
Do you have some initial perf numbers? It seems fine to me to remain
row-based inside Spark with whole-stage-codegen, and convert rows to
columnar batches when communicating with external systems.
On Mon, Mar 25, 2019 at 1:05 PM Bobby Evans wrote:
> This thread is to discuss adding in support
Spark 2.0 is EOL. Can you try 2.3 or 2.4?
On Thu, Mar 21, 2019 at 10:23 AM asma zgolli wrote:
> Hello ,
>
> I need to cross my data and i'm executing a cross join on two dataframes .
>
> C = A.crossJoin(B)
> A has 50 records
> B has 5 records
>
> the result im getting with spark 2.0 is a
Try `val enconder = RowEncoder(df.schema).resolveAndBind()` ?
On Thu, Mar 21, 2019 at 5:39 PM Long, Andrew
wrote:
> Thanks a ton for the help!
>
>
>
> Is there a standardized way of converting the internal row to rows?
>
>
>
> I’ve tried this but im getting an exception
>
>
>
> *val *enconder =
Did you include the whole error message?
On Fri, Mar 22, 2019 at 12:45 AM 563280193 <563280...@qq.com> wrote:
> Hi ,
> I ran a spark sql like this:
>
> *select imei,tag, product_id,*
> * sum(case when succ1>=1 then 1 else 0 end) as succ,*
> * sum(case when fail1>=1 and succ1=0 then 1
cool, thanks!
On Sat, Mar 16, 2019 at 1:08 AM shane knapp wrote:
> well, that box rebooted in record time! we're back up and building.
>
> and as always, i'll keep a close eye on things today... jenkins usually
> works great, until it doesn't. :\
>
> On Fri, Mar 15, 2019 at 9:52 AM shane
Which version of Parquet has this bug? Maybe we can downgrade it.
On Mon, Mar 11, 2019 at 10:34 AM Mark Hamstra
wrote:
> It worked in 2.3. We broke it with 2.4.0 and were informed of that
> regression late in the 2.4.0 release process. Since we didn't fix it before
> the 2.4.0 release, it
+1
On Sat, Mar 2, 2019 at 6:11 AM Yinan Li wrote:
> +1
>
> On Fri, Mar 1, 2019 at 12:37 PM Tom Graves
> wrote:
>
>> +1 for the SPIP.
>>
>> Tom
>>
>> On Friday, March 1, 2019, 8:14:43 AM CST, Xingbo Jiang <
>> jiangxb1...@gmail.com> wrote:
>>
>>
>> Hi all,
>>
>> I want to call for a vote of
+1, thanks for making it clear that this SPIP focuses on
high-level direction!
On Sat, Mar 2, 2019 at 9:35 AM Reynold Xin wrote:
> Thanks Ryan. +1.
>
>
>
>
> On Fri, Mar 01, 2019 at 5:33 PM, Ryan Blue wrote:
>
>> Actually, I went ahead and removed the confusing section. There is no
>> public
it in 3.0.
>> >
>> > On Tue, Feb 26, 2019 at 5:10 PM Matt Cheah wrote:
>> > Will that then require an API break down the line? Do we save that for
>> Spark 4?
>> >
>> >
>> >
>> >
>> > -Matt Cheah?
>> >
>> >
I think this is the right direction to go, but I'm wondering how can Spark
support these new types if the underlying data sources(like parquet files)
do not support them yet.
I took a quick look at the new doc for file formats, but not sure what's
the proposal. Are we going to implement these new
Thanks Shane!
On Wed, Feb 20, 2019 at 6:48 AM shane knapp wrote:
> alright, i increased the httpd and proxy timeouts and kicked apache. i'll
> keep an eye on things, but as of right now we're happily building.
>
> On Tue, Feb 19, 2019 at 2:25 PM shane knapp wrote:
>
>> aand i had to issue
+1
On Tue, Feb 19, 2019 at 10:50 AM Ryan Blue
wrote:
> Hi everyone,
>
> It looks like there is consensus on the proposal, so I'd like to start a
> vote thread on the SPIP for identifiers in multi-catalog Spark.
>
> The doc is available here:
>
I think this is the right direction to go. Shall we move forward with a
vote and detailed designs?
On Mon, Feb 4, 2019 at 9:57 AM Ryan Blue wrote:
> Hi everyone,
>
> This is a follow-up to the "Identifiers with multi-catalog support"
> discussion thread. I've taken the proposal I posted to that
great job!
On Mon, Feb 18, 2019 at 4:24 PM Hyukjin Kwon wrote:
> Yay! Good job Takeshi!
>
> On Mon, 18 Feb 2019, 14:47 Takeshi Yamamuro
>> We are happy to announce the availability of Spark 2.3.3!
>>
>> Apache Spark 2.3.3 is a maintenance release, based on the branch-2.3
>> maintenance branch
in the Spark SQL example, `year("1912")` means, first cast "1912" to date
type, and then call the "year" function.
in the Postgres example, `date_part('year',TIMESTAMP '2017')` means, get a
timestamp literal, and call the "date_part" function.
Can you try date literal in Postgres?
On Mon, Feb
Do you know which bug ORC 1.5.2 introduced? Or is it because Hive uses a
legacy version of ORC which has a bug?
On Thu, Feb 14, 2019 at 2:35 PM Darcy Shen wrote:
>
> We found that ORC table created by Spark 2.4 failed to be read by Hive
> 2.1.1.
>
>
> spark-sql -e 'CREATE TABLE tmp.orcTable2
+1 for 2.4.1
On Tue, Feb 12, 2019 at 7:55 PM Hyukjin Kwon wrote:
> +1 for 2.4.1
>
> 2019년 2월 12일 (화) 오후 4:56, Dongjin Lee 님이 작성:
>
>> > SPARK-23539 is a non-trivial improvement, so probably would not be
>> back-ported to 2.4.x.
>>
>> Got it. It seems reasonable.
>>
>> Committers:
>>
>> Please
what do you mean by ''Tungsten Consumer"?
On Fri, Feb 8, 2019 at 6:11 PM Jack Kolokasis
wrote:
> Hello all,
> I am studying about Tungsten Project and I am wondering when Spark
> creates a Tungsten consumer. While I am running some applications, I see
> that Spark creates Tungsten Consumer
d like to discuss it and hopefully we can get a consensus to move forward.
Thanks,
Wenchen
On Thu, Jan 10, 2019 at 2:40 AM Ryan Blue wrote:
> Hi everyone,
>
> This is a quick reminder that there is a DSv2 community sync tonight at 5
> PM PST. These community syncs are open to
PM Wenchen Fan wrote:
> I'm OK with it, i.e. fail the write if there are negative-scale decimals
> (we need to document it though). We can improve it later in data source v2.
>
> On Mon, Jan 7, 2019 at 10:09 PM Marco Gaido
> wrote:
>
>> In general we can say that som
+1
On Wed, Jan 9, 2019 at 3:37 AM DB Tsai wrote:
> +1
>
> Sincerely,
>
> DB Tsai
> --
> Web: https://www.dbtsai.com
> PGP Key ID: 0x5CED8B896A6BDFA0
>
> On Tue, Jan 8, 2019 at 11:14 AM Dongjoon Hyun
> wrote:
> >
> > Please vote on
; Il giorno lun 7 gen 2019 alle ore 15:03 Wenchen Fan
> ha scritto:
>
>> AFAIK parquet spec says decimal scale can't be negative. If we want to
>> officially support negative-scale decimal, we should clearly define the
>> behavior when writing negative-scale decimals to parq
something better, like casting decimal(1, -20) to decimal(20, 0)
before writing.
On Mon, Jan 7, 2019 at 9:32 PM Marco Gaido wrote:
> Hi Wenchen,
>
> thanks for your email. I agree adding doc for decimal type, but I am not
> sure what you mean speaking of the behavior when
I think we need to do this for backward compatibility, and according to the
discussion in the doc, SQL standard allows negative scale.
To do this, I think the PR should also include a doc for the decimal type,
like the definition of precision and scale(this one
IIRC there was a change to the release process: we stop using the shared
gpg key on Jenkins, but use the personal key of the release manager. I'm
not sure Jenkins can help testing package anymore.
BTW release manager needs to run the packaging script by himself. If there
is a problem, the release
Streaming interactively? etc.
On Sat, Dec 22, 2018 at 3:04 PM JackyLee wrote:
> Hi wenchen
> I have been working at SQLStreaming for a year, and I have promoted it
> in company.
> I have seen the design for Kafka or the Calcite, and I believe my
> design
> is better than
check status/progress of a streaming query?
4. ...
Basically, we should check what functions the DataStreamReader/Writer API
support, and see if we can support it with SQL as well.
Thanks for your proposal!
Wenchen
On Mon, Oct 22, 2018 at 11:15 AM JackyLee wrote:
> The code of SQLStreaming
end-user to
>> than having a consistent behavior between native clients and Spark clients.
>> This is why I asked if the goal was to just have a common "Spark" behavior
>> because I don't think it makes sense if you consider multiple interaction
>> points f
syntax. Isn't the right thing (for
> option 1) to pass the default through to the underlying data source?
> Sources that don't support defaults would throw an exception.
>
> On Wed, Dec 19, 2018 at 6:29 PM Wenchen Fan wrote:
>
>> The standard ADD COLUMN SQL syntax is: ALTER
+1, at least it should only send one email when a PR is merged.
On Thu, Dec 20, 2018 at 10:58 AM Nicholas Chammas <
nicholas.cham...@gmail.com> wrote:
> Can we somehow disable these new email alerts coming through for the Spark
> website repo?
>
> On Wed, Dec 19, 2018 at 8:25 PM GitBox wrote:
>
or not missing columns are
> OK and let the Datasource deal with the missing data based on it's
> underlying storage.
>
> On Wed, Dec 19, 2018 at 8:23 AM Wenchen Fan wrote:
>
>> I agree that we should not rewrite existing parquet files when a new
>> column is added, but we should a
I agree that we should not rewrite existing parquet files when a new column
is added, but we should also try out best to make the behavior same as
RDBMS/SQL standard.
1. it should be the user who decides the default value of a column, by
CREATE TABLE, or ALTER TABLE ADD COLUMN, or ALTER TABLE
I like this proposal.
> We'll get agreement across Spark, Hive, and Impala.
Shall we include Parquet and ORC? If they don't support it, it's hard for
general query engines like Spark to support it.
On Wed, Dec 12, 2018 at 3:36 AM Li Jin wrote:
> Of course. I added some comments in the doc.
>
IIRC, the reason we did it is: `SQLConf` was in SQL core module. So we need
to create methods in `CatalystConf`, and `SQLConf` implements
`CatalystConf`.
Now the problem has gone: we moved `SQLConf` to catalyst module. I think we
can remove these methods.
On Fri, Dec 14, 2018 at 3:45 PM Reynold
expressions/functions can be expensive and I do think Spark should trust
data source and not re-apply pushed filters. If data source lies, many
things can go wrong...
On Sun, Dec 9, 2018 at 8:17 PM Jörn Franke wrote:
> Well even if it has to apply it again, if pushdown is activated then it
>
great job! thanks a lot!
On Thu, Dec 6, 2018 at 9:39 AM Hyukjin Kwon wrote:
> It's merged now and in developer tools page -
> http://spark.apache.org/developer-tools.html#individual-tests
> Have some func with PySpark testing!
>
> 2018년 12월 5일 (수) 오후 4:30, Hyukjin Kwon 님이 작성:
>
>> Hey all, I
The bucket feature is designed to only work with data sources with table
support, and currently the table support is not public yet, which means no
external data sources can access bucketing information right now. The
bucket feature only works with Spark native file source tables.
We are working
be used in release notes.
Thanks,
Wenchen
On Wed, Nov 28, 2018 at 12:54 PM JackyLee wrote:
> +1
>
> Please add me to the Google Hangout invite.
>
>
>
> --
> Sent from: http://apache-spark-developers-li
es are
meant to be private. We should make it consistent between MiMa and doc
generator though.
On Wed, Nov 14, 2018 at 10:41 AM Marcelo Vanzin wrote:
> On Tue, Nov 13, 2018 at 6:26 PM Wenchen Fan wrote:
> > Recently I updated the MiMa exclusion rules, and found MiMa tracks some
>
tation.
I'm sending this email to collect more feedback, and hope we can come up
with a clear definition about what is "private".
Thanks,
Wenchen
batch interfaces. We can delay the branching
further, but that needs some refactoring of the continuous streaming data
source APIs.
I think the capability API is not a must-have at the current stage, but
it's worth to investigate further and see which use cases it can help.
Thanks,
Wenchen
On S
_
> > From: Sean Owen
> > Sent: Tuesday, November 6, 2018 10:51 AM
> > To: Shivaram Venkataraman
> > Cc: Felix Cheung; Wenchen Fan; Matei Zaharia; dev
> > Subject: Re: [CRAN-pretest-archived] CRAN submission SparkR 2.4.0
> >
> > I think the second optio
To view the release notes:
https://spark.apache.org/releases/spark-release-2-4-0.html
Thanks,
Wenchen
PS: If you see any issues with the release notes, webpage or published
artifacts, please contact me directly off-list.
+ user list
On Fri, Nov 9, 2018 at 2:20 AM Wenchen Fan wrote:
> resend
>
> On Thu, Nov 8, 2018 at 11:02 PM Wenchen Fan wrote:
>
>>
>>
>> -- Forwarded message -
>> From: Wenchen Fan
>> Date: Thu, Nov 8, 2018 at 10:55 PM
>> S
Actually I did it 3 hours ago, however the mail server seems to have some
problems and my email was lost. Let me resend it.
On Fri, Nov 9, 2018 at 1:56 AM Reynold Xin wrote:
> The website is already up but I didn’t see any email announcement yet.
>
ping
On Fri, Nov 9, 2018 at 2:20 AM Wenchen Fan wrote:
> Actually I did it 3 hours ago, however the mail server seems to have some
> problems and my email was lost. Let me resend it.
>
> On Fri, Nov 9, 2018 at 1:56 AM Reynold Xin wrote:
>
>> The website is already up but I
resend
On Thu, Nov 8, 2018 at 11:02 PM Wenchen Fan wrote:
>
>
> -- Forwarded message -
> From: Wenchen Fan
> Date: Thu, Nov 8, 2018 at 10:55 PM
> Subject: [ANNOUNCE] Announcing Apache Spark 2.4.0
> To: Spark dev list
>
>
> Hi all,
>
> Apa
We make Scala 2.11 the default one in Spark 2.0, then drop Scala 2.10 in
Spark 2.3. Shall we follow it and drop Scala 2.11 at some point of Spark
3.x?
On Wed, Nov 7, 2018 at 8:55 AM Reynold Xin wrote:
> Have we deprecated Scala 2.11 already in an existing release?
>
> On Tue, Nov 6, 2018 at
Do you mean we should have a 2.4.0 release without CRAN and then do a 2.4.1
immediately?
On Wed, Nov 7, 2018 at 12:34 AM Felix Cheung
wrote:
> Shivaram and I were discussing.
> Actually we worked with them before. Another possible approach is to
> remove the vignettes eval and all test from the
Have you tried Maven instead of SBT? This looks like a Java dependency
problem, e.g. a wrong version of Avro is picked.
On Tue, Nov 6, 2018 at 8:30 AM shane knapp wrote:
> i'm really close (for real: really close!) on the ubuntu port... but one
> build has been a thorn in my side and i was
This vote passes! I'll follow up with a formal release announcement soon.
+1:
Xiao Li (binding)
Sean Owen (binding)
Gengliang Wang
Hyukjin Kwon
Wenchen Fan (binding)
Ryan Blue
Bryan Cutler
Marcelo Vanzin (binding)
Reynold Xin (binding)
Chitral Verma
Dilip Biswal
Denny Lee
Felix Cheung (binding
be a pycodestyle or flake8 version issue?
>>> On Wed, Oct 31, 2018 at 7:43 AM Wenchen Fan wrote:
>>> >
>>> > The Jenkins job spark-master-lint keeps failing. The error message is
>>> > flake8.exceptions.FailedToLoadPlugin: Flake8 failed
The Jenkins job spark-master-lint keeps failing. The error message is
flake8.exceptions.FailedToLoadPlugin: Flake8 failed to load plugin
"pycodestyle.break_after_binary_operator" due to 'module' object has no
attribute 'break_after_binary_operator'.
flake8 checks failed.
As an example please see
API
Migrate existing data sources
Move data source v2 and built-in implementations to individual modules.
Let's have more discussion over the hangout.
Thanks,
Wenchen
On Tue, Oct 30, 2018 at 4:32 AM Ryan Blue wrote:
> Everyone,
>
> There are now 25 guests invited, which is a lot
gt;> >> >
>> >> > These items are still targeted to 2.4.0; Xiangrui I assume these
>> >> > should just be untargeted now, or resolved?
>> >> > SPARK-25584 Document libsvm data source in doc site
>> >> > SPARK-25346 Document Spark b
Please vote on releasing the following candidate as Apache Spark version
2.4.0.
The vote is open until November 1 PST and passes if a majority +1 PMC votes
are cast, with
a minimum of 3 +1 votes.
[ ] +1 Release this package as Apache Spark 2.4.0
[ ] -1 Do not release this package because ...
To
Friday at the bay area is Saturday at my side, it will be great if we can
pick a day from Monday to Thursday.
On Fri, Oct 26, 2018 at 8:08 AM Ryan Blue wrote:
> Since not many people have replied with a time window, how about we aim
> for 5PM PDT? That should work for Wenchen and most
Big +1 on this!
I live in UTC+8 and I'm available from 8 am, which is 5 pm in the bay area.
Hopefully we can coordinate a time that fits everyone.
Thanks
Wenchen
On Fri, Oct 26, 2018 at 7:21 AM Dongjoon Hyun
wrote:
> +1. Thank you for volunteering, Ryan!
>
> Bests,
> Dongjoon.
Any updates on this topic? https://github.com/apache/spark/pull/22827 is
merged and 2.4 is unblocked.
I'll cut RC5 shortly after the weekend, and it will be great to include the
change proposed here.
Thanks,
Wenchen
On Fri, Oct 26, 2018 at 12:55 AM Stavros Kontopoulos <
stavros.konto
a wrong result due to our undefined behavior. I agree we
>> should revert the newly added map-oriented functions. In 3.0 release, we
>> need to define the behavior of duplicate keys in the data type MAP and fix
>> all the related issues that are confusing to our end users.
>
t
they are invisible to end-users, and the weird behavior of Spark map type
with duplicated keys are not escalated. We should fix it ASAP in the master
branch.
If others are OK with it, I'll start a new RC after that PR is merged.
Thanks,
Wenchen
On Thu, Oct 25, 2018 at 10:32 AM Dongjoon Hyun
wro
rk-sql> select map(1,2,1,3); // Spark 2.4.0 RC4
> {1:3}
>
>
> hive> select map(1,2,1,3); // Hive 1.2.2
> OK
> {1:3}
>
>
> presto> SELECT map_concat(map(array[1],array[2]), map(array[1],array[3]));
> // Presto 0.212
> _col0
> ---
> {1=3}
>
>
> Bes
t to master branch.
Going back to https://issues.apache.org/jira/browse/SPARK-25823, it's just
a symptom of the hive map value converter bug. I think it's a non-blocker.
Thanks,
Wenchen
On Thu, Oct 25, 2018 at 5:31 AM Dongjoon Hyun
wrote:
> Hi, All.
>
> -0 due to the foll
nd call it the same release.
>>
>> I'd still like some confirmation that someone can build and pass tests
>> with -Pkubernetes, maybe? It actually all passed with the 2.11 build.
>> I don't think it's a 2.12 incompatibility, but rather than the K8S
>> tests maybe don't quite w
This is logically `sum( if(isnull(col), 0, col) )` right?
On Tue, Oct 23, 2018 at 2:58 PM 陶 加涛 wrote:
> The name is from Apache Calcite, And it doesn’t matter, we can introduce
> our own.
>
>
>
>
>
> ---
>
> Regards!
>
> Aron Tao
>
>
>
> *发件人**: *Mark Hamstra
> *日期**: *2018年10月23日 星期二 12:28
>
t; >> at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
> >> at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
> >> at org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.org
> $scalatest$BeforeAndAfter$$super$ru
Since GitHub and Jenkins are in a chaotic state, I didn't wait for a green
Jenkins QA job for the RC4 commit. We should fail this RC if the Jenkins is
broken (very unlikely).
I'm adding my own +1, all known blockers are resolved.
On Tue, Oct 23, 2018 at 1:42 AM Wenchen Fan wrote:
> Please v
Please vote on releasing the following candidate as Apache Spark version
2.4.0.
The vote is open until October 26 PST and passes if a majority +1 PMC votes
are cast, with
a minimum of 3 +1 votes.
[ ] +1 Release this package as Apache Spark 2.4.0
[ ] -1 Do not release this package because ...
To
AFAIK we haven't tested Java 9+ yet, so I'm ok to change it.
Hi Dongjoon can you make a PR for it? We can merge it very soon if we
decide to do it.
Thanks,
Wenchen
On Sat, Oct 20, 2018 at 5:27 AM Dongjoon Hyun
wrote:
> From the document, should we be more specific with 'Java 8' inst
think this is a corner case. Who will develop a data source
like that? If users do have this requirement, maybe they can just implement
a table with empty schema, and in Spark the append operator skips input
schema validation if table schema is empty.
Any thoughts?
Thanks,
Wenchen
On Thu, Sep 20
olution.
>
> how can we insert a coalesce as a new map-phase (new job on application
> manager with narrow dependency) instead of modifying the existing reduce
> phase? i am saying map-phase because it should not introduce a new shuffle:
> this is wasteful and unnecessary.
>
>
&
8 partitions and 2048 tasks and then a map
> phase with 10 partitions and 10 tasks that writes to hdfs?
>
> every time i try to do this using coalesce the shuffle ends up having 10
> tasks which is unacceptable due to OOM. this makes coalesce somewhat
> useless.
>
> On Wed, Oc
Note that, it was deprecated in 2.3.0 already:
https://spark.apache.org/docs/2.3.0/streaming-flume-integration.html
On Fri, Oct 12, 2018 at 12:46 AM Reynold Xin wrote:
> Sounds like a good idea...
>
> > On Oct 11, 2018, at 6:40 PM, Sean Owen wrote:
> >
> > Yep, that already exists as Bahir.
>
source, schema.toAttributes, options, ident, userSpecifiedSchema)
>>
>> }
>>
>>
>>
>> Correct this?
>>
>>
>>
>> Or even creating a new create which simply gets the schema as non
>> optional?
>>
>>
>>
>
Forgot to cc dev-list
-- Forwarded message -
From: Wenchen Fan
Date: Thu, Oct 11, 2018 at 10:14 AM
Subject: Re: [VOTE] SPARK 2.4.0 (RC3)
To:
Cc: Sean Owen
Ah sorry guys, I just copy-paste the voting email from the last RC and
forgot to update the date :P
The voting should
I'm adding my own +1, since there are no known blocker issues. The
correctness issue has been fixed, the streaming Java API problem has been
resolved, and we have upgraded to Scala 2.12.7.
On Thu, Oct 11, 2018 at 12:46 AM Wenchen Fan wrote:
> Please vote on releasing the following candid
Please vote on releasing the following candidate as Apache Spark version
2.4.0.
The vote is open until October 1 PST and passes if a majority +1 PMC votes
are cast, with
a minimum of 3 +1 votes.
[ ] +1 Release this package as Apache Spark 2.4.0
[ ] -1 Do not release this package because ...
To
Note that, RDD partitions and Spark tasks are not always 1-1 mapping.
Assuming `rdd1` has 100 partitions, and `rdd2 = rdd1.coalesce(10)`. Then
`rdd2` has 10 partitions, and there is no shuffle between `rdd1` and
`rdd2`. During scheduling, `rdd1` and `rdd2` are in the same stage, and
this stage
301 - 400 of 547 matches
Mail list logo