[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-07-24 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/21474


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-07-20 Thread jerryshao
Github user jerryshao commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r204021872
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
--- End diff --

I think the original purpose to set to `Long.MaxValue` is to avoid using 
this configuration by default, user should set to a proper size to enable this 
feature. But anyway I think the current change is also fine.


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-06-28 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r198891670
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
+  // fetch-to-mem is guaranteed to fail if the message is bigger than 
2 GB, so we might
+  // as well use fetch-to-disk in that case.  The message includes 
some metadata in addition
+  // to the block data itself (in particular UploadBlock has a lot of 
metadata), so we leave
+  // extra room.
+  .createWithDefault(Int.MaxValue - 500)
--- End diff --

sounds good, updated


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-06-27 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r198709276
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
+  // fetch-to-mem is guaranteed to fail if the message is bigger than 
2 GB, so we might
+  // as well use fetch-to-disk in that case.  The message includes 
some metadata in addition
+  // to the block data itself (in particular UploadBlock has a lot of 
metadata), so we leave
+  // extra room.
+  .createWithDefault(Int.MaxValue - 500)
--- End diff --

Actually I prefer 512 to 500  :)


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-06-02 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r192551649
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
+  // fetch-to-mem is guaranteed to fail if the message is bigger than 
2 GB, so we might
+  // as well use fetch-to-disk in that case.  The message includes 
some metadata in addition
+  // to the block data itself (in particular UploadBlock has a lot of 
metadata), so we leave
+  // extra room.
+  .createWithDefault(Int.MaxValue - 500)
--- End diff --

right, it's the default and the user can change the value to leave more 
margin.


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-06-01 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r192487033
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
+  // fetch-to-mem is guaranteed to fail if the message is bigger than 
2 GB, so we might
+  // as well use fetch-to-disk in that case.  The message includes 
some metadata in addition
+  // to the block data itself (in particular UploadBlock has a lot of 
metadata), so we leave
+  // extra room.
+  .createWithDefault(Int.MaxValue - 500)
--- End diff --

no guarantee its big enough.  Seemed OK in the test I tried.  But 
UploadBlock has some variable length strings so can't say for sure.

I'm fine making this much bigger, eg. 1 MB -- you'd only be bigger than 
that with a pathological case.  then there would be *some* cases where we'd be 
taking an old message which was fine with fetch-to-mem and we'd switch to 
fetch-to-disk.  But such a tiny case, and not an unreasonable change even for 
that ... so should be OK.


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-05-31 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/21474#discussion_r192295318
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/config/package.scala ---
@@ -429,7 +429,11 @@ package object config {
 "external shuffle service, this feature can only be worked when 
external shuffle" +
 "service is newer than Spark 2.2.")
   .bytesConf(ByteUnit.BYTE)
-  .createWithDefault(Long.MaxValue)
+  // fetch-to-mem is guaranteed to fail if the message is bigger than 
2 GB, so we might
+  // as well use fetch-to-disk in that case.  The message includes 
some metadata in addition
+  // to the block data itself (in particular UploadBlock has a lot of 
metadata), so we leave
+  // extra room.
+  .createWithDefault(Int.MaxValue - 500)
--- End diff --

is the fixed buffer of 500 bytes enough for metadata?


---

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



[GitHub] spark pull request #21474: [SPARK-24297][CORE] Fetch-to-disk by default for ...

2018-05-31 Thread squito
GitHub user squito opened a pull request:

https://github.com/apache/spark/pull/21474

[SPARK-24297][CORE] Fetch-to-disk by default for > 2gb

Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB,
so we might as well use fetch-to-disk in that case.  The message includes
some metadata in addition to the block data itself (in particular
UploadBlock has a lot of metadata), so we leave a little room.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/squito/spark SPARK-24297

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21474.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21474


commit 6b7c4f716e13cc4dcde756e98478edffb9e0d0f5
Author: Imran Rashid 
Date:   2018-05-31T21:35:01Z

[SPARK-24297][CORE] Fetch-to-disk by default for > 2gb

Fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB,
so we might as well use fetch-to-disk in that case.  The message includes
some metadata in addition to the block data itself (in particular
UploadBlock has a lot of metadata), so we leave a little room.




---

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