Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
31 changes: 28 additions & 3 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.util

import java.io._
import java.lang.{Byte => JByte}
import java.lang.InternalError
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
Expand Down Expand Up @@ -2052,6 +2051,30 @@ private[spark] object Utils extends Logging {
}
}

/**
* Implements the same logic as JDK `java.lang.String#trim` by removing leading and trailing
* non-printable characters less or equal to '\u0020' (SPACE) but preserves natural line
* delimiters according to [[java.util.Properties]] load method. The natural line delimiters are
* removed by JDK during load. Therefore any remaining ones have been specifically provided and
* escaped by the user, and must not be ignored
*
* @param str
* @return the trimmed value of str
*/
private[util] def trimExceptCRLF(str: String): String = {
val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch =>
ch > ' ' || ch == '\r' || ch == '\n'
}

val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter)
val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter)
if (firstPos >= 0 && lastPos >= 0) {
str.substring(firstPos, lastPos + 1)
} else {
""
}
}

/** Load properties present in the given file. */
def getPropertiesFromFile(filename: String): Map[String, String] = {
val file = new File(filename)
Expand All @@ -2062,8 +2085,10 @@ private[spark] object Utils extends Logging {
try {
val properties = new Properties()
properties.load(inReader)
properties.stringPropertyNames().asScala.map(
k => (k, properties.getProperty(k).trim)).toMap
properties.stringPropertyNames().asScala
.map { k => (k, trimExceptCRLF(properties.getProperty(k))) }
.toMap

} catch {
case e: IOException =>
throw new SparkException(s"Failed when loading Spark properties from $filename", e)
Expand Down
47 changes: 47 additions & 0 deletions core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1144,6 +1144,53 @@ class SparkSubmitSuite
conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}")
conf1.get("spark.submit.pyFiles") should (startWith("/"))
}

test("handles natural line delimiters in --properties-file and --conf uniformly") {
val delimKey = "spark.my.delimiter."
val LF = "\n"
val CR = "\r"

val lineFeedFromCommandLine = s"${delimKey}lineFeedFromCommandLine" -> LF
val leadingDelimKeyFromFile = s"${delimKey}leadingDelimKeyFromFile" -> s"${LF}blah"
val trailingDelimKeyFromFile = s"${delimKey}trailingDelimKeyFromFile" -> s"blah${CR}"
val infixDelimFromFile = s"${delimKey}infixDelimFromFile" -> s"${CR}blah${LF}"
val nonDelimSpaceFromFile = s"${delimKey}nonDelimSpaceFromFile" -> " blah\f"

val testProps = Seq(leadingDelimKeyFromFile, trailingDelimKeyFromFile, infixDelimFromFile,
nonDelimSpaceFromFile)

val props = new java.util.Properties()
val propsFile = File.createTempFile("test-spark-conf", ".properties",
Utils.createTempDir())
val propsOutputStream = new FileOutputStream(propsFile)
try {
testProps.foreach { case (k, v) => props.put(k, v) }
props.store(propsOutputStream, "test whitespace")
} finally {
propsOutputStream.close()
}

val clArgs = Seq(
"--class", "org.SomeClass",
"--conf", s"${lineFeedFromCommandLine._1}=${lineFeedFromCommandLine._2}",
"--conf", "spark.master=yarn",
"--properties-file", propsFile.getPath,
"thejar.jar")

val appArgs = new SparkSubmitArguments(clArgs)
val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs)

Seq(
lineFeedFromCommandLine,
leadingDelimKeyFromFile,
trailingDelimKeyFromFile,
infixDelimFromFile
).foreach { case (k, v) =>
conf.get(k) should be (v)
}

conf.get(nonDelimSpaceFromFile._1) should be ("blah")
}
}

object SparkSubmitSuite extends SparkFunSuite with TimeLimits {
Expand Down
28 changes: 28 additions & 0 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1205,6 +1205,34 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
assert(Utils.stringHalfWidth("\u0967\u0968\u0969") == 3)
// scalastyle:on nonascii
}

test("trimExceptCRLF standalone") {
val crlfSet = Set("\r", "\n")
val nonPrintableButCRLF = (0 to 32).map(_.toChar.toString).toSet -- crlfSet

// identity for CRLF
crlfSet.foreach { s => Utils.trimExceptCRLF(s) === s }

// empty for other non-printables
nonPrintableButCRLF.foreach { s => assert(Utils.trimExceptCRLF(s) === "") }

// identity for a printable string
assert(Utils.trimExceptCRLF("a") === "a")

// identity for strings with CRLF
crlfSet.foreach { s =>
assert(Utils.trimExceptCRLF(s"${s}a") === s"${s}a")
assert(Utils.trimExceptCRLF(s"a${s}") === s"a${s}")
assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b")
}

// trim nonPrintableButCRLF except when inside a string
nonPrintableButCRLF.foreach { s =>
assert(Utils.trimExceptCRLF(s"${s}a") === "a")
assert(Utils.trimExceptCRLF(s"a${s}") === "a")
assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b")
}
}
}

private class SimpleExtension
Expand Down