Re: [ANNOUNCE] Announcing Apache Spark 2.4.4

2019-09-01 Thread Hyukjin Kwon
YaY!

2019년 9월 2일 (월) 오후 1:27, Wenchen Fan 님이 작성:

> Great! Thanks!
>
> On Mon, Sep 2, 2019 at 5:55 AM Dongjoon Hyun 
> wrote:
>
>> We are happy to announce the availability of Spark 2.4.4!
>>
>> Spark 2.4.4 is a maintenance release containing stability fixes. This
>> release is based on the branch-2.4 maintenance branch of Spark. We
>> strongly
>> recommend all 2.4 users to upgrade to this stable release.
>>
>> To download Spark 2.4.4, head over to the download page:
>> http://spark.apache.org/downloads.html
>>
>> Note that you might need to clear your browser cache or
>> to use `Private`/`Incognito` mode according to your browsers.
>>
>> To view the release notes:
>> https://spark.apache.org/releases/spark-release-2-4-4.html
>>
>> We would like to acknowledge all community members for contributing to
>> this
>> release. This release would not have been possible without you.
>>
>> Dongjoon Hyun
>>
>


Re: [ANNOUNCE] Announcing Apache Spark 2.4.4

2019-09-01 Thread Wenchen Fan
Great! Thanks!

On Mon, Sep 2, 2019 at 5:55 AM Dongjoon Hyun 
wrote:

> We are happy to announce the availability of Spark 2.4.4!
>
> Spark 2.4.4 is a maintenance release containing stability fixes. This
> release is based on the branch-2.4 maintenance branch of Spark. We strongly
> recommend all 2.4 users to upgrade to this stable release.
>
> To download Spark 2.4.4, head over to the download page:
> http://spark.apache.org/downloads.html
>
> Note that you might need to clear your browser cache or
> to use `Private`/`Incognito` mode according to your browsers.
>
> To view the release notes:
> https://spark.apache.org/releases/spark-release-2-4-4.html
>
> We would like to acknowledge all community members for contributing to this
> release. This release would not have been possible without you.
>
> Dongjoon Hyun
>


Re: concurrent writes with dynamic partition overwrite mode

2019-09-01 Thread Koert Kuipers
https://issues.apache.org/jira/browse/SPARK-28945

On Sun, Sep 1, 2019 at 2:53 PM Koert Kuipers  wrote:

> hi,
> i am struggling to understand if concurrent writes to same basedir but
> different partitions are save with file sources such as parquet.
>
> i tested this in spark 2.4 and spark 3.0.0-SNAPSHOT with real concurrent
> jobs on hdfs and it seemed to work fine. admittedly this was a rather
> limited test.
> as the jobs are running i see on hdfs:
>
> drwxr-xr-x   - koert koert  0 2019-08-29 18:18
> out/.spark-staging-be40030e-8eef-4680-85ac-b55e6519df60/partition=2
> Found 1 items
> drwxr-xr-x   - koert koert  0 2019-08-29 18:18
> out/.spark-staging-d25f16d3-8f2d-4cf4-89bd-09256469b5e5/partition=1
> Found 1 items
> drwxr-xr-x   - koert koert  0 2019-08-29 18:17 out/_temporary/0
>
> it seems each job has its own temporary directory it writes to, set to
> ".spark-staging-" + jobId
> this is consistent with the stagingDir i found in
> org.apache.spark.internal.io.HadoopMapReduceCommitProtocol, and i can find
> it being used specifically for dynamic partition overwrite mode.
> so it thought with this i am all set.
> i dont really know what this _temporary/0 is for but it did not seem to
> get in the way.
>
> but then i added some unit tests that also do concurrent writes to
> different partitions with dynamic partition overwrite mode (these test are
> much more rigorous than my ad-hoc tests on hdfs), and now i see errors like
> this:
>
> java.io.FileNotFoundException: File file:/some/base/dir/_temporary/0 does
> not exist
> at
> org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:376)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1485)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1525)
> at
> org.apache.hadoop.fs.ChecksumFileSystem.listStatus(ChecksumFileSystem.java:570)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1485)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1525)
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.getAllCommittedTaskPaths(FileOutputCommitter.java:269)
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:309)
> at
> org.apache.parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:48)
> at
> org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.commitJob(HadoopMapReduceCommitProtocol.scala:166)
> at
> org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:215)
> at
> org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:172)
> at
> org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104)
> at
> org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102)
> at
> org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:123)
> at
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:168)
> at
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:192)
> at
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> at
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:189)
> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:164)
> at
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:105)
> at
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:105)
> at
> org.apache.spark.sql.DataFrameWriter.$anonfun$runCommand$1(DataFrameWriter.scala:718)
> at
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100)
> at
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
> at
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87)
> at
> org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:718)
> at
> org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:330)
> at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:314)
> at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:235)
> at com.tresata.spark.sql.source.ParquetSource.writeBatchImpl
>
> so this seems to hint that the issue is with that _temporary/0. why is it
> trying to do a listStatus on this _temporary/0?
> did i always have this issue and was just lucky enough to not run into it
> on hdfs, or is it specific to RawLocalFileSystem?
>
> thanks!
> koert
>


Re: [DISCUSSION]JDK11 for Apache 2.x?

2019-09-01 Thread Sean Owen
That's just the Oracle 'commercial' JDK though. OpenJDK is still OSS
and 8 is updated there too. Oracle JDK is still free too, just not for
commercial use and commercial support for 8 (IIUC). I think that's the
'audience' here, and I don't know how large or small that is.

That said a key first question is simply, how much breaking change is
necessary to get 2.x to work on JDK 11? If it's large I think that
might resolve the question right there. SPARK-24417 has most but not
necessarily all necessary changes under it.

Heck it could be maintained as a separate community project too, if
there is both interest from some parties to make it happen, but not
enough to commit to maintain all the backports and updated from 3.x.

On Sun, Sep 1, 2019 at 9:04 PM Holden Karau  wrote:
>
> So I'm not a lawyer, but looking at the JDK8 License FAQ ( 
> https://www.oracle.com/technetwork/java/javase/overview/oracle-jdk-faqs.html 
> ) makes me feel like we should support folks who are on the 2.x branch and 
> want to migrate away from JDK8. Although the Redhat post about OpenJDK8 is 
> reassuring ( 
> https://developers.redhat.com/blog/2019/05/22/openjdk-8-and-11-still-in-safe-hands/
>  ).
>
> On Tue, Aug 27, 2019 at 2:22 PM Sean Owen  wrote:
>>
>> Spark 3 will not require Java 11; it will work with Java 8 too. I
>> think the question is whether someone who _wants_ Java 11 should have
>> a 2.x release instead of 3.0.
>>
>> In practice... how much are people moving off Java 8 right now? It's
>> still my default, because most but not all things work with 11. (Here,
>> I'm using 11 as shorthand for 9+; it's the LTS release in the 9 to 11
>> series).
>> I get that the issue is support, but it seems that Oracle is still
>> providing public updates through Dec 2020. I think it's that
>> commercial support is now for-pay? And OpenJDK 8 still gets updates?
>> that I'm unclear on.
>>
>> Not that those aren't legitimate concerns, I'm just trying to figure
>> out how much need vs want there is and why out there.
>>
>>
>> On Tue, Aug 27, 2019 at 4:15 PM Jean Georges Perrin  wrote:
>> >
>> > Not a contributor, but a user perspective…
>> >
>> > As Spark 3.x will be an evolution, I am not completely shocked that it 
>> > would imply a Java 11 requirement as well. Would be great to have both 
>> > Java 8 and Java 11, but one needs to be able to say goodbye. Java 8 is 
>> > great, still using it actively in production, but we know its time is 
>> > limited, so, by the time we evolve to Spark 3, we could combine it with 
>> > Java 11.
>> >
>> > On the other hand, not everybody may think this way and it may slow down 
>> > the adoption of Spark 3…
>> >
>> > However, I concur with Sean, I don’t think another 2.x is needed for Java 
>> > 11.
>> >
>> > > On Aug 27, 2019, at 3:09 PM, Sean Owen  wrote:
>> > >
>> > > I think one of the key problems here are the required dependency
>> > > upgrades. It would mean many minor breaking changes and a few bigger
>> > > ones, notably around Hive, and forces a scala 2.12-only update. I
>> > > think my question is whether that even makes sense as a minor release?
>> > > it wouldn't be backwards compatible with 2.4 enough to call it a
>> > > low-risk update. It would be a smaller step than moving all the way to
>> > > 3.0, sure. I am not super against it, but we have to keep in mind how
>> > > much work it would then be to maintain two LTS 2.x releases, 2.4 and
>> > > the sort-of-compatible 2.5, while proceeding with 3.x.
>> > >
>> > > On Tue, Aug 27, 2019 at 2:01 PM DB Tsai  wrote:
>> > >>
>> > >> Hello everyone,
>> > >>
>> > >> Thank you all for working on supporting JDK11 in Apache Spark 3.0 as a 
>> > >> community.
>> > >>
>> > >> Java 8 is already end of life for commercial users, and many companies 
>> > >> are moving to Java 11.
>> > >> The release date for Apache Spark 3.0 is still not there yet, and there 
>> > >> are many API
>> > >> incompatibility issues when upgrading from Spark 2.x. As a result, 
>> > >> asking users to move to
>> > >> Spark 3.0 to use JDK 11 is not realistic.
>> > >>
>> > >> Should we backport PRs for JDK11 and cut a release in 2.x to support 
>> > >> JDK11?
>> > >>
>> > >> Should we cut a new Apache Spark 2.5 since the patches involve some of 
>> > >> the dependencies changes
>> > >> which is not desired in minor release?
>> > >>
>> > >> Thanks.
>> > >>
>> > >> DB Tsai  |  Siri Open Source Technologies [not a contribution]  |   
>> > >> Apple, Inc
>> > >>
>> > >>
>> > >> -
>> > >> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>> > >>
>> > >
>> > > -
>> > > To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>> > >
>> >
>> >
>> > -
>> > To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>> >
>>
>> 

Re: [DISCUSSION]JDK11 for Apache 2.x?

2019-09-01 Thread Holden Karau
So I'm not a lawyer, but looking at the JDK8 License FAQ (
https://www.oracle.com/technetwork/java/javase/overview/oracle-jdk-faqs.html )
makes me feel like we should support folks who are on the 2.x branch and
want to migrate away from JDK8. Although the Redhat post about OpenJDK8 is
reassuring (
https://developers.redhat.com/blog/2019/05/22/openjdk-8-and-11-still-in-safe-hands/
 ).

On Tue, Aug 27, 2019 at 2:22 PM Sean Owen  wrote:

> Spark 3 will not require Java 11; it will work with Java 8 too. I
> think the question is whether someone who _wants_ Java 11 should have
> a 2.x release instead of 3.0.
>
> In practice... how much are people moving off Java 8 right now? It's
> still my default, because most but not all things work with 11. (Here,
> I'm using 11 as shorthand for 9+; it's the LTS release in the 9 to 11
> series).
> I get that the issue is support, but it seems that Oracle is still
> providing public updates through Dec 2020. I think it's that
> commercial support is now for-pay? And OpenJDK 8 still gets updates?
> that I'm unclear on.
>
> Not that those aren't legitimate concerns, I'm just trying to figure
> out how much need vs want there is and why out there.
>
>
> On Tue, Aug 27, 2019 at 4:15 PM Jean Georges Perrin  wrote:
> >
> > Not a contributor, but a user perspective…
> >
> > As Spark 3.x will be an evolution, I am not completely shocked that it
> would imply a Java 11 requirement as well. Would be great to have both Java
> 8 and Java 11, but one needs to be able to say goodbye. Java 8 is great,
> still using it actively in production, but we know its time is limited, so,
> by the time we evolve to Spark 3, we could combine it with Java 11.
> >
> > On the other hand, not everybody may think this way and it may slow down
> the adoption of Spark 3…
> >
> > However, I concur with Sean, I don’t think another 2.x is needed for
> Java 11.
> >
> > > On Aug 27, 2019, at 3:09 PM, Sean Owen  wrote:
> > >
> > > I think one of the key problems here are the required dependency
> > > upgrades. It would mean many minor breaking changes and a few bigger
> > > ones, notably around Hive, and forces a scala 2.12-only update. I
> > > think my question is whether that even makes sense as a minor release?
> > > it wouldn't be backwards compatible with 2.4 enough to call it a
> > > low-risk update. It would be a smaller step than moving all the way to
> > > 3.0, sure. I am not super against it, but we have to keep in mind how
> > > much work it would then be to maintain two LTS 2.x releases, 2.4 and
> > > the sort-of-compatible 2.5, while proceeding with 3.x.
> > >
> > > On Tue, Aug 27, 2019 at 2:01 PM DB Tsai 
> wrote:
> > >>
> > >> Hello everyone,
> > >>
> > >> Thank you all for working on supporting JDK11 in Apache Spark 3.0 as
> a community.
> > >>
> > >> Java 8 is already end of life for commercial users, and many
> companies are moving to Java 11.
> > >> The release date for Apache Spark 3.0 is still not there yet, and
> there are many API
> > >> incompatibility issues when upgrading from Spark 2.x. As a result,
> asking users to move to
> > >> Spark 3.0 to use JDK 11 is not realistic.
> > >>
> > >> Should we backport PRs for JDK11 and cut a release in 2.x to support
> JDK11?
> > >>
> > >> Should we cut a new Apache Spark 2.5 since the patches involve some
> of the dependencies changes
> > >> which is not desired in minor release?
> > >>
> > >> Thanks.
> > >>
> > >> DB Tsai  |  Siri Open Source Technologies [not a contribution]  |  
> Apple, Inc
> > >>
> > >>
> > >> -
> > >> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
> > >>
> > >
> > > -
> > > To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
> > >
> >
> >
> > -
> > To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
> >
>
> -
> To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
>
>

-- 
Twitter: https://twitter.com/holdenkarau
Books (Learning Spark, High Performance Spark, etc.):
https://amzn.to/2MaRAG9  
YouTube Live Streams: https://www.youtube.com/user/holdenkarau


Re: Design review of SPARK-28594

2019-09-01 Thread Jungtaek Lim
Great, thanks for reviewing, Felix!

On Mon, Sep 2, 2019 at 2:16 AM Felix Cheung 
wrote:

> I did review it and solving this problem makes sense. I will comment in
> the JIRA.
>
> --
> *From:* Jungtaek Lim 
> *Sent:* Sunday, August 25, 2019 3:34:22 PM
> *To:* dev 
> *Subject:* Design review of SPARK-28594
>
> Hi devs,
>
> I have been working on designing SPARK-28594 [1] (though I've started with
> this via different requests) and design doc is now available [2].
>
> Let me describe SPARK-28954 briefly - single and growing event log file
> for application has been major issue for streaming application as as long
> as event log just grows while the application is running, and lots of
> issues occur from there. The only viable workaround has been disabling
> event log which is not easily acceptable. Maybe stopping the application
> and rerunning would be another approach but it sounds really odd to stop
> the application due to event log. SPARK-28594 enables the way to roll the
> event log files, with compacting old event log files without losing the
> ability to replay whole logs.
>
> While I'll break down issue into subtask and start from easier one, in
> parallel I'd like to ask for reviewing on the design to get better idea and
> find possible defects of design.
>
> Please note that the doc is intended to describe the detailed changes
> (closer to the implementation details) and is not a kind of SPIP because I
> wouldn't feel going through SPIP process for this improvement - the change
> would be rather not huge and the proposal works orthogonal to current
> feature. Please let me know if it's not the case and SPIP process is
> necessary.
>
> Thanks,
> Jungtaek Lim (HeartSaVioR)
>
> 1. https://issues.apache.org/jira/browse/SPARK-28594
> 2.
> https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing
>
>

-- 
Name : Jungtaek Lim
Blog : http://medium.com/@heartsavior
Twitter : http://twitter.com/heartsavior
LinkedIn : http://www.linkedin.com/in/heartsavior


[ANNOUNCE] Announcing Apache Spark 2.4.4

2019-09-01 Thread Dongjoon Hyun
We are happy to announce the availability of Spark 2.4.4!

Spark 2.4.4 is a maintenance release containing stability fixes. This
release is based on the branch-2.4 maintenance branch of Spark. We strongly
recommend all 2.4 users to upgrade to this stable release.

To download Spark 2.4.4, head over to the download page:
http://spark.apache.org/downloads.html

Note that you might need to clear your browser cache or
to use `Private`/`Incognito` mode according to your browsers.

To view the release notes:
https://spark.apache.org/releases/spark-release-2-4-4.html

We would like to acknowledge all community members for contributing to this
release. This release would not have been possible without you.

Dongjoon Hyun


concurrent writes with dynamic partition overwrite mode

2019-09-01 Thread Koert Kuipers
hi,
i am struggling to understand if concurrent writes to same basedir but
different partitions are save with file sources such as parquet.

i tested this in spark 2.4 and spark 3.0.0-SNAPSHOT with real concurrent
jobs on hdfs and it seemed to work fine. admittedly this was a rather
limited test.
as the jobs are running i see on hdfs:

drwxr-xr-x   - koert koert  0 2019-08-29 18:18
out/.spark-staging-be40030e-8eef-4680-85ac-b55e6519df60/partition=2
Found 1 items
drwxr-xr-x   - koert koert  0 2019-08-29 18:18
out/.spark-staging-d25f16d3-8f2d-4cf4-89bd-09256469b5e5/partition=1
Found 1 items
drwxr-xr-x   - koert koert  0 2019-08-29 18:17 out/_temporary/0

it seems each job has its own temporary directory it writes to, set to
".spark-staging-" + jobId
this is consistent with the stagingDir i found in
org.apache.spark.internal.io.HadoopMapReduceCommitProtocol, and i can find
it being used specifically for dynamic partition overwrite mode.
so it thought with this i am all set.
i dont really know what this _temporary/0 is for but it did not seem to get
in the way.

but then i added some unit tests that also do concurrent writes to
different partitions with dynamic partition overwrite mode (these test are
much more rigorous than my ad-hoc tests on hdfs), and now i see errors like
this:

java.io.FileNotFoundException: File file:/some/base/dir/_temporary/0 does
not exist
at
org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:376)
at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1485)
at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1525)
at
org.apache.hadoop.fs.ChecksumFileSystem.listStatus(ChecksumFileSystem.java:570)
at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1485)
at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1525)
at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.getAllCommittedTaskPaths(FileOutputCommitter.java:269)
at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:309)
at
org.apache.parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:48)
at
org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.commitJob(HadoopMapReduceCommitProtocol.scala:166)
at
org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:215)
at
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:172)
at
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104)
at
org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102)
at
org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:123)
at
org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:168)
at
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:192)
at
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at
org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:189)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:164)
at
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:105)
at
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:105)
at
org.apache.spark.sql.DataFrameWriter.$anonfun$runCommand$1(DataFrameWriter.scala:718)
at
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100)
at
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
at
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87)
at
org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:718)
at
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:330)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:314)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:235)
at com.tresata.spark.sql.source.ParquetSource.writeBatchImpl

so this seems to hint that the issue is with that _temporary/0. why is it
trying to do a listStatus on this _temporary/0?
did i always have this issue and was just lucky enough to not run into it
on hdfs, or is it specific to RawLocalFileSystem?

thanks!
koert


Re: Design review of SPARK-28594

2019-09-01 Thread Felix Cheung
I did review it and solving this problem makes sense. I will comment in the 
JIRA.


From: Jungtaek Lim 
Sent: Sunday, August 25, 2019 3:34:22 PM
To: dev 
Subject: Design review of SPARK-28594

Hi devs,

I have been working on designing SPARK-28594 [1] (though I've started with this 
via different requests) and design doc is now available [2].

Let me describe SPARK-28954 briefly - single and growing event log file for 
application has been major issue for streaming application as as long as event 
log just grows while the application is running, and lots of issues occur from 
there. The only viable workaround has been disabling event log which is not 
easily acceptable. Maybe stopping the application and rerunning would be 
another approach but it sounds really odd to stop the application due to event 
log. SPARK-28594 enables the way to roll the event log files, with compacting 
old event log files without losing the ability to replay whole logs.

While I'll break down issue into subtask and start from easier one, in parallel 
I'd like to ask for reviewing on the design to get better idea and find 
possible defects of design.

Please note that the doc is intended to describe the detailed changes (closer 
to the implementation details) and is not a kind of SPIP because I wouldn't 
feel going through SPIP process for this improvement - the change would be 
rather not huge and the proposal works orthogonal to current feature. Please 
let me know if it's not the case and SPIP process is necessary.

Thanks,
Jungtaek Lim (HeartSaVioR)

1. https://issues.apache.org/jira/browse/SPARK-28594
2. 
https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing