[jira] [Updated] (SPARK-33206) Spark Shuffle Index Cache calculates memory usage wrong

2020-10-21 Thread Lars Francke (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-33206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Francke updated SPARK-33206:
-
Description: 
SPARK-21501 changed the spark shuffle index service to be based on memory 
instead of the number of files.

Unfortunately, there's a problem with the calculation which is based on size 
information provided by `ShuffleIndexInformation`.

It is based purely on the file size of the cached file on disk.

We're running in OOMs with very small index files (byte size ~16 bytes) but the 
overhead of the ShuffleIndexInformation around this is much larger (e.g. 184 
bytes, see screenshot). We need to take this into account and should probably 
add a fixed overhead of somewhere between 152 and 180 bytes according to my 
tests. I'm not 100% sure what the correct number is and it'll also depend on 
the architecture etc. so we can't be exact anyway.

If we do that we can maybe get rid of the size field in ShuffleIndexInformation 
to save a few more bytes per entry.

In effect this means that for small files we use up about 70-100 times as much 
memory as we intend to. Our NodeManagers OOM with 4GB and more of 
indexShuffleCache.

 

 

  was:
dSPARK-21501 changed the spark shuffle index service to be based on memory 
instead of the number of files.

Unfortunately, there's a problem with the calculation which is based on size 
information provided by `ShuffleIndexInformation`.

It is based purely on the file size of the cached file on disk.

We're running in OOMs with very small index files (byte size ~16 bytes) but the 
overhead of the ShuffleIndexInformation around this is much larger (e.g. 184 
bytes, see screenshot). We need to take this into account and should probably 
add a fixed overhead of somewhere between 152 and 180 bytes according to my 
tests. I'm not 100% sure what the correct number is and it'll also depend on 
the architecture etc. so we can't be exact anyway.

If we do that we can maybe get rid of the size field in ShuffleIndexInformation 
to save a few more bytes per entry.

In effect this means that for small files we use up about 70-100 times as much 
memory as we intend to. Our NodeManagers OOM with 4GB and more of 
indexShuffleCache.

 

 


> Spark Shuffle Index Cache calculates memory usage wrong
> ---
>
> Key: SPARK-33206
> URL: https://issues.apache.org/jira/browse/SPARK-33206
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 2.4.0, 3.0.1
>Reporter: Lars Francke
>Priority: Major
> Attachments: image001(1).png
>
>
> SPARK-21501 changed the spark shuffle index service to be based on memory 
> instead of the number of files.
> Unfortunately, there's a problem with the calculation which is based on size 
> information provided by `ShuffleIndexInformation`.
> It is based purely on the file size of the cached file on disk.
> We're running in OOMs with very small index files (byte size ~16 bytes) but 
> the overhead of the ShuffleIndexInformation around this is much larger (e.g. 
> 184 bytes, see screenshot). We need to take this into account and should 
> probably add a fixed overhead of somewhere between 152 and 180 bytes 
> according to my tests. I'm not 100% sure what the correct number is and it'll 
> also depend on the architecture etc. so we can't be exact anyway.
> If we do that we can maybe get rid of the size field in 
> ShuffleIndexInformation to save a few more bytes per entry.
> In effect this means that for small files we use up about 70-100 times as 
> much memory as we intend to. Our NodeManagers OOM with 4GB and more of 
> indexShuffleCache.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-33206) Spark Shuffle Index Cache calculates memory usage wrong

2020-10-21 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-33206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17218306#comment-17218306
 ] 

Lars Francke commented on SPARK-33206:
--

I used YourKit (thank you for the free license!) and it claims that 
ShuffleIndexInformation uses 152 byte of retained memory when it caches a 0 
byte file.

> Spark Shuffle Index Cache calculates memory usage wrong
> ---
>
> Key: SPARK-33206
> URL: https://issues.apache.org/jira/browse/SPARK-33206
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 2.4.0, 3.0.1
>Reporter: Lars Francke
>Priority: Major
> Attachments: image001(1).png
>
>
> dSPARK-21501 changed the spark shuffle index service to be based on memory 
> instead of the number of files.
> Unfortunately, there's a problem with the calculation which is based on size 
> information provided by `ShuffleIndexInformation`.
> It is based purely on the file size of the cached file on disk.
> We're running in OOMs with very small index files (byte size ~16 bytes) but 
> the overhead of the ShuffleIndexInformation around this is much larger (e.g. 
> 184 bytes, see screenshot). We need to take this into account and should 
> probably add a fixed overhead of somewhere between 152 and 180 bytes 
> according to my tests. I'm not 100% sure what the correct number is and it'll 
> also depend on the architecture etc. so we can't be exact anyway.
> If we do that we can maybe get rid of the size field in 
> ShuffleIndexInformation to save a few more bytes per entry.
> In effect this means that for small files we use up about 70-100 times as 
> much memory as we intend to. Our NodeManagers OOM with 4GB and more of 
> indexShuffleCache.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-33206) Spark Shuffle Index Cache calculates memory usage wrong

2020-10-21 Thread Lars Francke (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-33206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Francke updated SPARK-33206:
-
Description: 
dSPARK-21501 changed the spark shuffle index service to be based on memory 
instead of the number of files.

Unfortunately, there's a problem with the calculation which is based on size 
information provided by `ShuffleIndexInformation`.

It is based purely on the file size of the cached file on disk.

We're running in OOMs with very small index files (byte size ~16 bytes) but the 
overhead of the ShuffleIndexInformation around this is much larger (e.g. 184 
bytes, see screenshot). We need to take this into account and should probably 
add a fixed overhead of somewhere between 152 and 180 bytes according to my 
tests. I'm not 100% sure what the correct number is and it'll also depend on 
the architecture etc. so we can't be exact anyway.

If we do that we can maybe get rid of the size field in ShuffleIndexInformation 
to save a few more bytes per entry.

In effect this means that for small files we use up about 70-100 times as much 
memory as we intend to. Our NodeManagers OOM with 4GB and more of 
indexShuffleCache.

 

 

  was:
SPARK-21501 changed the spark shuffle index service to be based on memory 
instead of the number of files.

Unfortunately, there's a problem with the calculation which is based on size 
information provided by `ShuffleIndexInformation`.

It is based purely on the file size of the cached file on disk.

We're running in OOMs with very small index files (byte size ~16 bytes) but the 
overhead of the ShuffleIndexInformation around this is much larger (e.g. 184 
bytes, see screenshot). We need to take this into account and should probably 
add a fixed overhead of somewhere between 152 and 180 bytes according to my 
tests. I'm not 100% sure what the correct number is and it'll also depend on 
the architecture etc. so we can't be exact anyway.

If we do that we can maybe get rid of the size field in ShuffleIndexInformation 
to save a few more bytes per entry.

In effect this means that for small files we use up about 70-100 times as much 
memory as we intend to. Our NodeManagers OOM with 4GB and more of 
indexShuffleCache.

 

 


> Spark Shuffle Index Cache calculates memory usage wrong
> ---
>
> Key: SPARK-33206
> URL: https://issues.apache.org/jira/browse/SPARK-33206
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 2.4.0, 3.0.1
>Reporter: Lars Francke
>Priority: Major
> Attachments: image001(1).png
>
>
> dSPARK-21501 changed the spark shuffle index service to be based on memory 
> instead of the number of files.
> Unfortunately, there's a problem with the calculation which is based on size 
> information provided by `ShuffleIndexInformation`.
> It is based purely on the file size of the cached file on disk.
> We're running in OOMs with very small index files (byte size ~16 bytes) but 
> the overhead of the ShuffleIndexInformation around this is much larger (e.g. 
> 184 bytes, see screenshot). We need to take this into account and should 
> probably add a fixed overhead of somewhere between 152 and 180 bytes 
> according to my tests. I'm not 100% sure what the correct number is and it'll 
> also depend on the architecture etc. so we can't be exact anyway.
> If we do that we can maybe get rid of the size field in 
> ShuffleIndexInformation to save a few more bytes per entry.
> In effect this means that for small files we use up about 70-100 times as 
> much memory as we intend to. Our NodeManagers OOM with 4GB and more of 
> indexShuffleCache.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-21501) Spark shuffle index cache size should be memory based

2020-10-21 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-21501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17218285#comment-17218285
 ] 

Lars Francke commented on SPARK-21501:
--

Just FYI for others stumbling across this: This has a bug in how the memory is 
calculated and might use way more than the 100MB it intends to.

See SPARK-33206 for details.

> Spark shuffle index cache size should be memory based
> -
>
> Key: SPARK-21501
> URL: https://issues.apache.org/jira/browse/SPARK-21501
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle, Spark Core
>Affects Versions: 2.1.0
>Reporter: Thomas Graves
>Assignee: Sanket Reddy
>Priority: Major
> Fix For: 2.3.0
>
>
> Right now the spark shuffle service has a cache for index files. It is based 
> on a # of files cached (spark.shuffle.service.index.cache.entries). This can 
> cause issues if people have a lot of reducers because the size of each entry 
> can fluctuate based on the # of reducers. 
> We saw an issues with a job that had 17 reducers and it caused NM with 
> spark shuffle service to use 700-800MB or memory in NM by itself.
> We should change this cache to be memory based and only allow a certain 
> memory size used. When I say memory based I mean the cache should have a 
> limit of say 100MB.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-33206) Spark Shuffle Index Cache calculates memory usage wrong

2020-10-21 Thread Lars Francke (Jira)


 [ 
https://issues.apache.org/jira/browse/SPARK-33206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Francke updated SPARK-33206:
-
Attachment: image001(1).png

> Spark Shuffle Index Cache calculates memory usage wrong
> ---
>
> Key: SPARK-33206
> URL: https://issues.apache.org/jira/browse/SPARK-33206
> Project: Spark
>  Issue Type: Bug
>  Components: Shuffle
>Affects Versions: 2.4.0, 3.0.1
>Reporter: Lars Francke
>Priority: Major
> Attachments: image001(1).png
>
>
> SPARK-21501 changed the spark shuffle index service to be based on memory 
> instead of the number of files.
> Unfortunately, there's a problem with the calculation which is based on size 
> information provided by `ShuffleIndexInformation`.
> It is based purely on the file size of the cached file on disk.
> We're running in OOMs with very small index files (byte size ~16 bytes) but 
> the overhead of the ShuffleIndexInformation around this is much larger (e.g. 
> 184 bytes, see screenshot). We need to take this into account and should 
> probably add a fixed overhead of somewhere between 152 and 180 bytes 
> according to my tests. I'm not 100% sure what the correct number is and it'll 
> also depend on the architecture etc. so we can't be exact anyway.
> If we do that we can maybe get rid of the size field in 
> ShuffleIndexInformation to save a few more bytes per entry.
> In effect this means that for small files we use up about 70-100 times as 
> much memory as we intend to. Our NodeManagers OOM with 4GB and more of 
> indexShuffleCache.
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-33206) Spark Shuffle Index Cache calculates memory usage wrong

2020-10-21 Thread Lars Francke (Jira)
Lars Francke created SPARK-33206:


 Summary: Spark Shuffle Index Cache calculates memory usage wrong
 Key: SPARK-33206
 URL: https://issues.apache.org/jira/browse/SPARK-33206
 Project: Spark
  Issue Type: Bug
  Components: Shuffle
Affects Versions: 3.0.1, 2.4.0
Reporter: Lars Francke
 Attachments: image001(1).png

SPARK-21501 changed the spark shuffle index service to be based on memory 
instead of the number of files.

Unfortunately, there's a problem with the calculation which is based on size 
information provided by `ShuffleIndexInformation`.

It is based purely on the file size of the cached file on disk.

We're running in OOMs with very small index files (byte size ~16 bytes) but the 
overhead of the ShuffleIndexInformation around this is much larger (e.g. 184 
bytes, see screenshot). We need to take this into account and should probably 
add a fixed overhead of somewhere between 152 and 180 bytes according to my 
tests. I'm not 100% sure what the correct number is and it'll also depend on 
the architecture etc. so we can't be exact anyway.

If we do that we can maybe get rid of the size field in ShuffleIndexInformation 
to save a few more bytes per entry.

In effect this means that for small files we use up about 70-100 times as much 
memory as we intend to. Our NodeManagers OOM with 4GB and more of 
indexShuffleCache.

 

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-30196) Bump lz4-java version to 1.7.0

2020-01-10 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-30196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17013039#comment-17013039
 ] 

Lars Francke commented on SPARK-30196:
--

Excellent, thank you!

> Bump lz4-java version to 1.7.0
> --
>
> Key: SPARK-30196
> URL: https://issues.apache.org/jira/browse/SPARK-30196
> Project: Spark
>  Issue Type: Improvement
>  Components: Build, Spark Core
>Affects Versions: 3.0.0
>Reporter: Takeshi Yamamuro
>Assignee: Takeshi Yamamuro
>Priority: Major
> Fix For: 3.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-30196) Bump lz4-java version to 1.7.0

2020-01-08 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-30196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17010475#comment-17010475
 ] 

Lars Francke commented on SPARK-30196:
--

Sorry I didn't have time to check yet but it seems as if [~maropu]. I'm back at 
work later this week and will be more responsive.

> Bump lz4-java version to 1.7.0
> --
>
> Key: SPARK-30196
> URL: https://issues.apache.org/jira/browse/SPARK-30196
> Project: Spark
>  Issue Type: Improvement
>  Components: Build, Spark Core
>Affects Versions: 3.0.0
>Reporter: Takeshi Yamamuro
>Assignee: Takeshi Yamamuro
>Priority: Major
> Fix For: 3.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-30196) Bump lz4-java version to 1.7.0

2019-12-29 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-30196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17005066#comment-17005066
 ] 

Lars Francke commented on SPARK-30196:
--

FYI: This seems to have broken Spark 3 on Mac OS for me due to
{code:java}
dyld: lazy symbol binding failed: Symbol not found: chkstk_darwin
  Referenced from: 
/private/var/folders/1v/ckh8py712_n_5r628_16w0l4gn/T/liblz4-java-820584040681098780.dylib
 (which was built for Mac OS X 10.15)
  Expected in: /usr/lib/libSystem.B.dylibdyld: Symbol not found: 
chkstk_darwin
  Referenced from: 
/private/var/folders/1v/ckh8py712_n_5r628_16w0l4gn/T/liblz4-java-820584040681098780.dylib
 (which was built for Mac OS X 10.15)
  Expected in: /usr/lib/libSystem.B.dylib
 {code}
 

I did a bit of googling but I'm not sure what's going on. Reverting to 1.6 
works for me. I'm on MacOS 10.13. Any hints are appreciated.

If the lz4 stuff really only works with MacOS 10.15 that'd be sad but I can't 
really believe that.
Has anyone tried Spark 3 Preview 2 on a Mac with 10.15/10.14/10.13?

> Bump lz4-java version to 1.7.0
> --
>
> Key: SPARK-30196
> URL: https://issues.apache.org/jira/browse/SPARK-30196
> Project: Spark
>  Issue Type: Improvement
>  Components: Build, Spark Core
>Affects Versions: 3.0.0
>Reporter: Takeshi Yamamuro
>Assignee: Takeshi Yamamuro
>Priority: Major
> Fix For: 3.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-29226) Upgrade jackson-databind to 2.9.10 and fix vulnerabilities.

2019-11-21 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-29226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16979868#comment-16979868
 ] 

Lars Francke commented on SPARK-29226:
--

Thanks for the quick follow-up. I did not create a Jira because I don't have 
enough time to follow up and check. SPARK-29483 looks like it could help. 
Unfortunately there's no indication whether it's in the 3.0.0-preview. Do you 
have any idea?

> Upgrade jackson-databind to 2.9.10 and fix vulnerabilities.
> ---
>
> Key: SPARK-29226
> URL: https://issues.apache.org/jira/browse/SPARK-29226
> Project: Spark
>  Issue Type: Dependency upgrade
>  Components: Build
>Affects Versions: 3.0.0
>Reporter: jiaan.geng
>Assignee: jiaan.geng
>Priority: Major
> Fix For: 3.0.0
>
>
> The current code uses com.fasterxml.jackson.core:jackson-databind:jar:2.9.9.3 
> and it will cause a security vulnerabilities. We could get some security info 
> from https://www.tenable.com/cve/CVE-2019-16335
> This reference remind to upgrate the version of `jackson-databind` to 2.9.10 
> or later.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-29226) Upgrade jackson-databind to 2.9.10 and fix vulnerabilities.

2019-11-21 Thread Lars Francke (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-29226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16979697#comment-16979697
 ] 

Lars Francke commented on SPARK-29226:
--

This causes the following exception for me:

 
{code:java}
Exception in thread "main" java.lang.NoClassDefFoundError: 
com/fasterxml/jackson/core/exc/InputCoercionExceptionException in thread "main" 
java.lang.NoClassDefFoundError: 
com/fasterxml/jackson/core/exc/InputCoercionException at 
com.fasterxml.jackson.databind.deser.BasicDeserializerFactory.createArrayDeserializer(BasicDeserializerFactory.java:1141)
 at 
com.fasterxml.jackson.databind.deser.DeserializerCache._createDeserializer2(DeserializerCache.java:372)
 at 
com.fasterxml.jackson.databind.deser.DeserializerCache._createDeserializer(DeserializerCache.java:349)
 at 
com.fasterxml.jackson.databind.deser.DeserializerCache._createAndCache2(DeserializerCache.java:264)
 at 
com.fasterxml.jackson.databind.deser.DeserializerCache._createAndCacheValueDeserializer(DeserializerCache.java:244)
 at 
com.fasterxml.jackson.databind.deser.DeserializerCache.findValueDeserializer(DeserializerCache.java:142)
 at 
com.fasterxml.jackson.databind.DeserializationContext.findRootValueDeserializer(DeserializationContext.java:476)
 at 
com.fasterxml.jackson.databind.ObjectMapper._findRootDeserializer(ObjectMapper.java:4389)
 at 
com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4198)
 at 
com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3205) 
at 
com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3173) 
at 
org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2.$anonfun$getPaths$1(FileDataSourceV2.scala:47)
 at scala.Option.map(Option.scala:230) at 
org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2.getPaths(FileDataSourceV2.scala:46)
 at 
org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2.getPaths$(FileDataSourceV2.scala:44)
 at 
org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2.getPaths(ParquetDataSourceV2.scala:26)
 at 
org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2.getTable(ParquetDataSourceV2.scala:33)
 at 
org.apache.spark.sql.DataFrameReader.$anonfun$load$1(DataFrameReader.scala:220) 
at scala.Option.map(Option.scala:230) at 
org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:206) at 
org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:674) at 
org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:658) at 
com.opencore.SparkReaderDemo.main(SparkReaderDemo.java:12)Caused by: 
java.lang.ClassNotFoundException: 
com.fasterxml.jackson.core.exc.InputCoercionException at 
java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:602)
 at 
java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
 at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:521) ... 23 more
{code}
This is because arrow brings in Jackson 2.7.9 and that seems to take precedence 
(at least when I run it from within IntelliJ). I'm not creating a new issue 
because I don't have enough time to investigate properly.

Declaring an explicit dependency on jackson-core in version 2.10.0 fixes this 
for me.

> Upgrade jackson-databind to 2.9.10 and fix vulnerabilities.
> ---
>
> Key: SPARK-29226
> URL: https://issues.apache.org/jira/browse/SPARK-29226
> Project: Spark
>  Issue Type: Dependency upgrade
>  Components: Build
>Affects Versions: 3.0.0
>Reporter: jiaan.geng
>Assignee: jiaan.geng
>Priority: Major
> Fix For: 3.0.0
>
>
> The current code uses com.fasterxml.jackson.core:jackson-databind:jar:2.9.9.3 
> and it will cause a security vulnerabilities. We could get some security info 
> from https://www.tenable.com/cve/CVE-2019-16335
> This reference remind to upgrate the version of `jackson-databind` to 2.9.10 
> or later.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-25390) data source V2 API refactoring

2019-06-25 Thread Lars Francke (JIRA)


[ 
https://issues.apache.org/jira/browse/SPARK-25390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16872684#comment-16872684
 ] 

Lars Francke commented on SPARK-25390:
--

Is there any kind of end-user documentation for this on how to use these APIs 
to develop custom sources?

When looking on the Spark homepage one only finds this documentation 
[https://spark.apache.org/docs/2.2.0/streaming-custom-receivers.html] it'd be 
useful to have a version of this for the new APIs

> data source V2 API refactoring
> --
>
> Key: SPARK-25390
> URL: https://issues.apache.org/jira/browse/SPARK-25390
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 3.0.0
>Reporter: Wenchen Fan
>Priority: Major
>
> Currently it's not very clear how we should abstract data source v2 API. The 
> abstraction should be unified between batch and streaming, or similar but 
> have a well-defined difference between batch and streaming. And the 
> abstraction should also include catalog/table.
> An example of the abstraction:
> {code}
> batch: catalog -> table -> scan
> streaming: catalog -> table -> stream -> scan
> {code}
> We should refactor the data source v2 API according to the abstraction



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-24049) Add a feature to not start speculative tasks before an absolute minimum amount of time has passed

2018-04-22 Thread Lars Francke (JIRA)
Lars Francke created SPARK-24049:


 Summary: Add a feature to not start speculative tasks before an 
absolute minimum amount of time has passed
 Key: SPARK-24049
 URL: https://issues.apache.org/jira/browse/SPARK-24049
 Project: Spark
  Issue Type: Improvement
  Components: Spark Core
Affects Versions: 2.3.0
Reporter: Lars Francke


Speculation currently has four different configuration options according to the 
docs:
* spark.speculation to turn it on or off
* .interval - how often it'll check
* .multiplier - how many times slower than average a task must be
* .quantile - fraction of tasks that have to be completed before speculation 
starts

What I'd love to see is a feature that allows me to set a minimum average 
duration. We are using a Jobserver to submit varying queries to Spark. Some of 
those have stages with tasks of an average length of ~20ms (yes, not idea but 
it happens). So with a multiplier of 2 a speculative task starts after 40ms. 
This happens quite a lot and in our use-case it doesn't make sense. We don't 
consider 40ms tasks "stragglers".

So it would make sense to add a parameter to only start speculation when the 
average task time is greater than X seconds/minutes/millis.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-24049) Add a feature to not start speculative tasks when average task duration is less than a configurable absolute number

2018-04-22 Thread Lars Francke (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-24049?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Francke updated SPARK-24049:
-
Summary: Add a feature to not start speculative tasks when average task 
duration is less than a configurable absolute number  (was: Add a feature to 
not start speculative tasks before an absolute minimum amount of time has 
passed)

> Add a feature to not start speculative tasks when average task duration is 
> less than a configurable absolute number
> ---
>
> Key: SPARK-24049
> URL: https://issues.apache.org/jira/browse/SPARK-24049
> Project: Spark
>  Issue Type: Improvement
>  Components: Spark Core
>Affects Versions: 2.3.0
>Reporter: Lars Francke
>Priority: Minor
>
> Speculation currently has four different configuration options according to 
> the docs:
> * spark.speculation to turn it on or off
> * .interval - how often it'll check
> * .multiplier - how many times slower than average a task must be
> * .quantile - fraction of tasks that have to be completed before speculation 
> starts
> What I'd love to see is a feature that allows me to set a minimum average 
> duration. We are using a Jobserver to submit varying queries to Spark. Some 
> of those have stages with tasks of an average length of ~20ms (yes, not idea 
> but it happens). So with a multiplier of 2 a speculative task starts after 
> 40ms. This happens quite a lot and in our use-case it doesn't make sense. We 
> don't consider 40ms tasks "stragglers".
> So it would make sense to add a parameter to only start speculation when the 
> average task time is greater than X seconds/minutes/millis.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Comment Edited] (SPARK-650) Add a "setup hook" API for running initialization code on each executor

2016-10-15 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15578914#comment-15578914
 ] 

Lars Francke edited comment on SPARK-650 at 10/15/16 11:22 PM:
---

I can only come up with three reasons at the moment. I hope they all make sense.

1) Singletons/Static Initialisers run once per Class Loader where this class is 
being loaded/used. I haven't actually seen this being a problem (and it might 
actually be desired behaviour in this case) but making the init step explicit 
would prevent this from ever becoming one.
2) I'd like to fail fast for some things and not upon first access (which might 
be behind a conditional somewhere)
3) It is hard enough to reason about where some piece of code is running as it 
is (Driver or Task/Executor), adding Singletons to the mix makes it even more 
confusing.


Thank you for reopening!


was (Author: lars_francke):
I can only come up with three reasons at the moment. I hope they all make sense.

1) Singletons/Static Initialisers run once per Class Loader where this class is 
being loaded/used. I haven't actually seen this being a problem (and it might 
actually be desired behaviour in this case) but making the init step explicit 
would prevent this from ever becoming one.
2) I'd like to fail fast for some things and not upon first access (which might 
be behind a conditional somewhere)
3) It is hard enough to reason about where some piece of code is running as it 
is (Driver or Task/Executor), adding Singletons to the mix makes it even more 
confusing.

> Add a "setup hook" API for running initialization code on each executor
> ---
>
> Key: SPARK-650
> URL: https://issues.apache.org/jira/browse/SPARK-650
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Reporter: Matei Zaharia
>Priority: Minor
>
> Would be useful to configure things like reporting libraries



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-650) Add a "setup hook" API for running initialization code on each executor

2016-10-15 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15578914#comment-15578914
 ] 

Lars Francke commented on SPARK-650:


I can only come up with three reasons at the moment. I hope they all make sense.

1) Singletons/Static Initialisers run once per Class Loader where this class is 
being loaded/used. I haven't actually seen this being a problem (and it might 
actually be desired behaviour in this case) but making the init step explicit 
would prevent this from ever becoming one.
2) I'd like to fail fast for some things and not upon first access (which might 
be behind a conditional somewhere)
3) It is hard enough to reason about where some piece of code is running as it 
is (Driver or Task/Executor), adding Singletons to the mix makes it even more 
confusing.

> Add a "setup hook" API for running initialization code on each executor
> ---
>
> Key: SPARK-650
> URL: https://issues.apache.org/jira/browse/SPARK-650
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Reporter: Matei Zaharia
>Priority: Minor
>
> Would be useful to configure things like reporting libraries



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-650) Add a "setup hook" API for running initialization code on each executor

2016-10-15 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15578875#comment-15578875
 ] 

Lars Francke commented on SPARK-650:


I also have to disagree with this being a duplicate or obsolete.

[~oarmand] and [~Skamandros] already mentioned reasons regarding the 
duplication.

About it being obsolete: I have seen multiple clients facing this problem, 
finding this issue and hoping it'd get fixed some day. I would hesitate a guess 
and say that most _users_ of Spark have no JIRA account here and do not 
register or log in just to vote for this issue. That said: This issue is (with 
six votes) in the top 150 out of almost 17k total issues in the Spark project.

As it happens this is a non-trivial thing to implement in Spark (as far as I 
can tell from my limited knowledge of the inner workings) so it's pretty hard 
for a "drive by" contributor to help here.

You had the discussion about community perception on the mailing list (re: 
Spark Improvement Proposals) and this issue happens to be one of those that at 
least I see popping up every once in a while in discussions with clients.

I would love to see this issue staying open as a feature request and have some 
hope that someone someday will implement it.

> Add a "setup hook" API for running initialization code on each executor
> ---
>
> Key: SPARK-650
> URL: https://issues.apache.org/jira/browse/SPARK-650
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Reporter: Matei Zaharia
>Priority: Minor
>
> Would be useful to configure things like reporting libraries



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-12177) Update KafkaDStreams to new Kafka 0.10 Consumer API

2016-06-21 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15341381#comment-15341381
 ] 

Lars Francke commented on SPARK-12177:
--

I have looked at the code and have only minor nit-picks about code-style. It 
built successfully for me as well and we hope to get it tested against a real 
cluster soon. I also managed to build it against Spark 1.6.1 with minor changes.

> Update KafkaDStreams to new Kafka 0.10 Consumer API
> ---
>
> Key: SPARK-12177
> URL: https://issues.apache.org/jira/browse/SPARK-12177
> Project: Spark
>  Issue Type: Improvement
>  Components: Streaming
>Affects Versions: 1.6.0
>Reporter: Nikita Tarasenko
>  Labels: consumer, kafka
>
> Kafka 0.9 already released and it introduce new consumer API that not 
> compatible with old one. So, I added new consumer api. I made separate 
> classes in package org.apache.spark.streaming.kafka.v09 with changed API. I 
> didn't remove old classes for more backward compatibility. User will not need 
> to change his old spark applications when he uprgade to new Spark version.
> Please rewiew my changes



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Updated] (SPARK-572) Forbid update of static mutable variables

2015-07-29 Thread Lars Francke (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-572?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lars Francke updated SPARK-572:
---
Description: 
Consider the following piece of code:
{code}
object Foo {
 var xx = -1

 def main() {
   xx = 1
   val sc = new SparkContext(...)
   sc.broadcast(xx)
   sc.parallelize(0 to 10).map(i={ ... xx ...})
 }
}
{code}
Can you guess the value of xx? It is 1 when you use the local scheduler and -1 
when you use the mesos scheduler. Given the complications, it should probably 
just be forbidden for now...

  was:
Consider the following piece of code:
pre
object Foo {
 var xx = -1

 def main() {
   xx = 1
   val sc = new SparkContext(...)
   sc.broadcast(xx)
   sc.parallelize(0 to 10).map(i={ ... xx ...})
 }
}
/pre
Can you guess the value of xx? It is 1 when you use the local scheduler and -1 
when you use the mesos scheduler. Given the complications, it should probably 
just be forbidden for now...


 Forbid update of static mutable variables
 -

 Key: SPARK-572
 URL: https://issues.apache.org/jira/browse/SPARK-572
 Project: Spark
  Issue Type: Improvement
Reporter: tjhunter

 Consider the following piece of code:
 {code}
 object Foo {
  var xx = -1
  def main() {
xx = 1
val sc = new SparkContext(...)
sc.broadcast(xx)
sc.parallelize(0 to 10).map(i={ ... xx ...})
  }
 }
 {code}
 Can you guess the value of xx? It is 1 when you use the local scheduler and 
 -1 when you use the mesos scheduler. Given the complications, it should 
 probably just be forbidden for now...



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-650) Add a setup hook API for running initialization code on each executor

2015-07-29 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-650?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14645919#comment-14645919
 ] 

Lars Francke commented on SPARK-650:


Not [~matei] but I think this would be a good idea to have. Abusing another 
undocumented concept doesn't seem like a nice way to treat a useful and common 
use-case.

 Add a setup hook API for running initialization code on each executor
 ---

 Key: SPARK-650
 URL: https://issues.apache.org/jira/browse/SPARK-650
 Project: Spark
  Issue Type: New Feature
  Components: Spark Core
Reporter: Matei Zaharia
Priority: Minor

 Would be useful to configure things like reporting libraries



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-2447) Add common solution for sending upsert actions to HBase (put, deletes, and increment)

2015-06-29 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-2447?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14605657#comment-14605657
 ] 

Lars Francke commented on SPARK-2447:
-

Hey Ted et. al,

thanks for the work on this. SparkOnHBase is super useful and clients are 
happily using it.

I wonder however what the future direction will be. Any progress on the 
question whether it's going to be integrated into Spark or not? I don't have a 
strong opinion either but I also don't feel that it would be _wrong_ to put it 
into core...

 Add common solution for sending upsert actions to HBase (put, deletes, and 
 increment)
 -

 Key: SPARK-2447
 URL: https://issues.apache.org/jira/browse/SPARK-2447
 Project: Spark
  Issue Type: New Feature
  Components: Spark Core, Streaming
Reporter: Ted Malaska
Assignee: Ted Malaska

 Going to review the design with Tdas today.  
 But first thoughts is to have an extension of VoidFunction that handles the 
 connection to HBase and allows for options such as turning auto flush off for 
 higher through put.
 Need to answer the following questions first.
 - Can it be written in Java or should it be written in Scala?
 - What is the best way to add the HBase dependency? (will review how Flume 
 does this as the first option)
 - What is the best way to do testing? (will review how Flume does this as the 
 first option)
 - How to support python? (python may be a different Jira it is unknown at 
 this time)
 Goals:
 - Simple to use
 - Stable
 - Supports high load
 - Documented (May be in a separate Jira need to ask Tdas)
 - Supports Java, Scala, and hopefully Python
 - Supports Streaming and normal Spark



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Created] (SPARK-8462) Documentation fixes for Spark SQL

2015-06-18 Thread Lars Francke (JIRA)
Lars Francke created SPARK-8462:
---

 Summary: Documentation fixes for Spark SQL
 Key: SPARK-8462
 URL: https://issues.apache.org/jira/browse/SPARK-8462
 Project: Spark
  Issue Type: Documentation
  Components: Documentation
Affects Versions: 1.4.0
Reporter: Lars Francke
Priority: Minor


This fixes various minor documentation issues on the Spark SQL page



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-7366) Support multi-line JSON objects

2015-05-05 Thread Lars Francke (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-7366?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14528489#comment-14528489
 ] 

Lars Francke commented on SPARK-7366:
-

This would be very helpful. ESRI has a RecordReader that could serve as 
inspiration as well. It does a reasonable job of finding boundaries on a JSON 
subset: 
https://github.com/Esri/spatial-framework-for-hadoop/blob/master/json/src/main/java/com/esri/json/hadoop/UnenclosedJsonRecordReader.java

 Support multi-line JSON objects
 ---

 Key: SPARK-7366
 URL: https://issues.apache.org/jira/browse/SPARK-7366
 Project: Spark
  Issue Type: Improvement
  Components: Input/Output
Reporter: Joe Halliwell
Priority: Minor

 h2. Background: why the existing formats aren't enough
 The present object-per-line format for ingesting JSON data has a couple of 
 deficiencies:
 1. It's not itself JSON
 2. It's often harder for humans to read
 The object-per-file format addresses these, but at a cost of producing many 
 files which can be unwieldy.
 Since it is feasible to read and write large JSON files via streaming (and 
 many systems do) it seems reasonable to support them directly as an input 
 format.
 h2. Suggested approach: use a depth hint
 The key challenge is to find record boundaries without parsing the file from 
 the start i.e. given an offset, locate a nearby boundary. In the general case 
 this is impossible: you can't be sure you've identified the start of a 
 top-level record without tracing back to the start of the file.
 However, if we know something more of the structure of the file i.e. maximum 
 object depth it seems plausible that we can do better.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org