[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-31 Thread asfgit
Github user asfgit closed the pull request at:

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


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229803309
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +473,42 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+
+  lazy val builderReflection: Option[(Class[_], Method, Method)] = Try {
+val cls = Utils.classForName(
+  
"org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder")
+(cls, cls.getMethod("replicate"), cls.getMethod("build"))
+  }.toOption
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding, 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
--- End diff --

"ec" is already explained in the line above. no need to repeat it.


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-31 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229802904
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +472,33 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding (ec), 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
+   * hflush(), hsync(), or append()
+   * 
https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations
+   */
+  // scalastyle:on line.size.limit
+  def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = {
+try {
+  // Use reflection as this uses apis only avialable in hadoop 3
+  val builderMethod = fs.getClass().getMethod("createFile", 
classOf[Path])
+  val builder = builderMethod.invoke(fs, path)
+  val builderCls = builder.getClass()
+  // this may throw a NoSuchMethodException if the path is not on hdfs
+  val replicateMethod = builderCls.getMethod("replicate")
+  val buildMethod = builderCls.getMethod("build")
+  val b2 = replicateMethod.invoke(builder)
+  buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream]
+} catch {
+  case  _: NoSuchMethodException =>
--- End diff --

If you get that exception, it's a proper error that shouldn't fall back to 
this code path.


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-31 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229577581
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +472,33 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding (ec), 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
+   * hflush(), hsync(), or append()
+   * 
https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations
+   */
+  // scalastyle:on line.size.limit
+  def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = {
+try {
+  // Use reflection as this uses apis only avialable in hadoop 3
+  val builderMethod = fs.getClass().getMethod("createFile", 
classOf[Path])
+  val builder = builderMethod.invoke(fs, path)
+  val builderCls = builder.getClass()
+  // this may throw a NoSuchMethodException if the path is not on hdfs
+  val replicateMethod = builderCls.getMethod("replicate")
+  val buildMethod = builderCls.getMethod("build")
+  val b2 = replicateMethod.invoke(builder)
+  buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream]
+} catch {
+  case  _: NoSuchMethodException =>
--- End diff --

Any chance of `InvocationTargetException`? 


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-30 Thread xiao-chen
Github user xiao-chen commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229403073
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +473,42 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+
+  lazy val builderReflection: Option[(Class[_], Method, Method)] = Try {
+val cls = Utils.classForName(
+  
"org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder")
+(cls, cls.getMethod("replicate"), cls.getMethod("build"))
+  }.toOption
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding, 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
+   * hflush(), hsync(), or append()
+   * 
https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations
+   */
+  // scalastyle:on line.size.limit
+  def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = {
+try {
+  // Use reflection as this uses apis only avialable in hadoop 3
+  val builderMethod = fs.getClass().getMethod("createFile", 
classOf[Path])
+  val builder = builderMethod.invoke(fs, path)
+  builderReflection match {
--- End diff --

FYI - I created https://issues.apache.org/jira/browse/HDFS-14038 for hadoop 
discussion.


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229172448
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +473,42 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+
+  lazy val builderReflection: Option[(Class[_], Method, Method)] = Try {
+val cls = Utils.classForName(
+  
"org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder")
+(cls, cls.getMethod("replicate"), cls.getMethod("build"))
+  }.toOption
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding, 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
+   * hflush(), hsync(), or append()
+   * 
https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations
+   */
+  // scalastyle:on line.size.limit
+  def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = {
+try {
+  // Use reflection as this uses apis only avialable in hadoop 3
+  val builderMethod = fs.getClass().getMethod("createFile", 
classOf[Path])
+  val builder = builderMethod.invoke(fs, path)
+  builderReflection match {
--- End diff --

good point on the reflection, I was trying something else in earlier 
experiments and didn't clean up.

on poking into `DistributedFileSystem` -- @xiao-chen had similar concerns, 
but also said it seemed there wasn't another option and it looked like an 
oversight in the hdfs api.  @steveloughran maybe you have thoughts here as well?


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229155491
  
--- Diff: docs/configuration.md ---
@@ -761,6 +761,17 @@ Apart from these, the following properties are also 
available, and may be useful
 Compression will use spark.io.compression.codec.
   
 
+
+  spark.eventLog.allowErasureCoding
+  false
+  
+Whether to allow event logs to use erasure coding, or turn erasure 
coding off, regardless of
+filesystem defaults.  On HDFS, erasure coded files will not update as 
quickly as regular
+replicated files, so they application updates will take longer to 
appear in the History Server.
+Note that even if this is true, spark will still not force the file to 
erasure coding, it will
--- End diff --

nit: `to erasure coding` -> `to use erasure coding`?


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229154733
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +473,42 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+
+  lazy val builderReflection: Option[(Class[_], Method, Method)] = Try {
+val cls = Utils.classForName(
+  
"org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder")
+(cls, cls.getMethod("replicate"), cls.getMethod("build"))
+  }.toOption
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding, 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
--- End diff --

nit: `ec` -> `erasure coding`


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229102664
  
--- Diff: docs/configuration.md ---
@@ -761,6 +761,17 @@ Apart from these, the following properties are also 
available, and may be useful
 Compression will use spark.io.compression.codec.
   
 
+
+  spark.eventLog.allowErasureCoding
+  false
+  
+Whether to allow event logs to use erasure coding, or turn erasure 
coding off, regardless of
+filesystem defaults.  On HDFS, erasure coded files will not update as 
quickly as regular
+replicated files, so they application updates will take longer to 
appear in the History Server.
+Note that even if this is true, spark will still not force the file to 
erasure coding, it will
--- End diff --

Spark


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229104197
  
--- Diff: docs/configuration.md ---
@@ -761,6 +761,17 @@ Apart from these, the following properties are also 
available, and may be useful
 Compression will use spark.io.compression.codec.
   
 
+
+  spark.eventLog.allowErasureCoding
+  false
+  
+Whether to allow event logs to use erasure coding, or turn erasure 
coding off, regardless of
+filesystem defaults.  On HDFS, erasure coded files will not update as 
quickly as regular
+replicated files, so they application updates will take longer to 
appear in the History Server.
--- End diff --

s/they/the


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229103471
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -26,11 +27,12 @@ import scala.collection.JavaConverters._
 import scala.collection.immutable.Map
 import scala.collection.mutable
 import scala.collection.mutable.HashMap
+import scala.util.Try
--- End diff --

Not used?


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/22881#discussion_r229102457
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
---
@@ -471,4 +473,42 @@ object SparkHadoopUtil {
   hadoopConf.set(key.substring("spark.hadoop.".length), value)
 }
   }
+
+
+  lazy val builderReflection: Option[(Class[_], Method, Method)] = Try {
+val cls = Utils.classForName(
+  
"org.apache.hadoop.hdfs.DistributedFileSystem$HdfsDataOutputStreamBuilder")
+(cls, cls.getMethod("replicate"), cls.getMethod("build"))
+  }.toOption
+
+  // scalastyle:off line.size.limit
+  /**
+   * Create a path that uses replication instead of erasure coding, 
regardless of the default
+   * configuration in hdfs for the given path.  This can be helpful as 
hdfs ec doesn't support
+   * hflush(), hsync(), or append()
+   * 
https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations
+   */
+  // scalastyle:on line.size.limit
+  def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = {
+try {
+  // Use reflection as this uses apis only avialable in hadoop 3
+  val builderMethod = fs.getClass().getMethod("createFile", 
classOf[Path])
+  val builder = builderMethod.invoke(fs, path)
+  builderReflection match {
--- End diff --

Not sure `builderReflection` is helping here. 

Using `builder.getClass().getMethod("replicate")` would achieve the same, 
no? And keep all related code here.

BTW it's not optimal to have to poke into `DistributedFileSystem` for this. 
That's marked as "limited private" and "unstable" in the hadoop libs. But there 
doesn't seem to be an alternative...


---

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



[GitHub] spark pull request #22881: [SPARK-25855][CORE] Don't use erasure coding for ...

2018-10-29 Thread squito
GitHub user squito opened a pull request:

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

[SPARK-25855][CORE] Don't use erasure coding for event logs by default

## What changes were proposed in this pull request?

This turns off hdfs erasure coding by default for event logs, regardless of 
filesystem defaults.  Because this requires apis only available in hadoop 3, 
this uses reflection.  EC isn't a very good choice for event logs, as hflush() 
is a no-op, and so updates to the file are not visible for a long time.  This 
can still be configured by setting "spark.eventLog.allowErasureCoding=true", 
which will use filesystem defaults.

## How was this patch tested?

deployed a cluster with the changes with HDFS EC on.  By default, event 
logs didn't use EC, but configuration still would allow EC.

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

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

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

https://github.com/apache/spark/pull/22881.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 #22881


commit 005ee5494acd3d9f0721ad24ba3700d8905e2e26
Author: Imran Rashid 
Date:   2018-10-26T19:03:43Z

[SPARK-25855][CORE][STREAMING] Don't use HDFS EC for event logs and WAL

hdfs erasure coding doesn't support hflush(), hsync(), or append(),
which doesn't work well for event logs and the WAL, so be sure we never
use it for those files, regardless of the configuration of hdfs.

commit 04b968a0223e195f1c7e6d6684274bd7f8484069
Author: Imran Rashid 
Date:   2018-10-26T20:22:11Z

fix

commit 8a9392c875b9b2aec048940a8ae7d03529bfc641
Author: Imran Rashid 
Date:   2018-10-29T15:56:20Z

make it configurable

commit cd28e61fe9232927ea66b3beb4af5c5d699bb6d3
Author: Imran Rashid 
Date:   2018-10-29T20:09:28Z

remove changes for WAL




---

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