Skip to content

Commit c99fc9a

Browse files
Marcelo Vanzingatorsmile
Marcelo Vanzin
authored andcommitted
[SPARK-23550][CORE] Cleanup Utils.
A few different things going on: - Remove unused methods. - Move JSON methods to the only class that uses them. - Move test-only methods to TestUtils. - Make getMaxResultSize() a config constant. - Reuse functionality from existing libraries (JRE or JavaUtils) where possible. The change also includes changes to a few tests to call `Utils.createTempFile` correctly, so that temp dirs are created under the designated top-level temp dir instead of potentially polluting git index. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #20706 from vanzin/SPARK-23550.
1 parent 53561d2 commit c99fc9a

File tree

21 files changed

+152
-236
lines changed

21 files changed

+152
-236
lines changed

core/src/main/scala/org/apache/spark/TestUtils.scala

Lines changed: 25 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.net.{HttpURLConnection, URI, URL}
2222
import java.nio.charset.StandardCharsets
2323
import java.security.SecureRandom
2424
import java.security.cert.X509Certificate
25-
import java.util.Arrays
25+
import java.util.{Arrays, Properties}
2626
import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit}
2727
import java.util.jar.{JarEntry, JarOutputStream}
2828
import javax.net.ssl._
@@ -35,6 +35,7 @@ import scala.sys.process.{Process, ProcessLogger}
3535
import scala.util.Try
3636

3737
import com.google.common.io.{ByteStreams, Files}
38+
import org.apache.log4j.PropertyConfigurator
3839

3940
import org.apache.spark.executor.TaskMetrics
4041
import org.apache.spark.scheduler._
@@ -256,6 +257,29 @@ private[spark] object TestUtils {
256257
s"Can't find $numExecutors executors before $timeout milliseconds elapsed")
257258
}
258259

260+
/**
261+
* config a log4j properties used for testsuite
262+
*/
263+
def configTestLog4j(level: String): Unit = {
264+
val pro = new Properties()
265+
pro.put("log4j.rootLogger", s"$level, console")
266+
pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender")
267+
pro.put("log4j.appender.console.target", "System.err")
268+
pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout")
269+
pro.put("log4j.appender.console.layout.ConversionPattern",
270+
"%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n")
271+
PropertyConfigurator.configure(pro)
272+
}
273+
274+
/**
275+
* Lists files recursively.
276+
*/
277+
def recursiveList(f: File): Array[File] = {
278+
require(f.isDirectory)
279+
val current = f.listFiles
280+
current ++ current.filter(_.isDirectory).flatMap(recursiveList)
281+
}
282+
259283
}
260284

261285

core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.deploy
1919

20-
import java.io.{ByteArrayOutputStream, PrintStream}
20+
import java.io.{ByteArrayOutputStream, File, PrintStream}
2121
import java.lang.reflect.InvocationTargetException
2222
import java.net.URI
2323
import java.nio.charset.StandardCharsets
@@ -233,7 +233,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
233233
// Set name from main class if not given
234234
name = Option(name).orElse(Option(mainClass)).orNull
235235
if (name == null && primaryResource != null) {
236-
name = Utils.stripDirectory(primaryResource)
236+
name = new File(primaryResource).getName()
237237
}
238238

239239
// Action should be SUBMIT unless otherwise specified

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
3535
import org.apache.spark._
3636
import org.apache.spark.deploy.SparkHadoopUtil
3737
import org.apache.spark.internal.Logging
38+
import org.apache.spark.internal.config._
3839
import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
3940
import org.apache.spark.rpc.RpcTimeout
4041
import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
@@ -141,8 +142,7 @@ private[spark] class Executor(
141142
conf.getSizeAsBytes("spark.task.maxDirectResultSize", 1L << 20),
142143
RpcUtils.maxMessageSizeBytes(conf))
143144

144-
// Limit of bytes for total size of results (default is 1GB)
145-
private val maxResultSize = Utils.getMaxResultSize(conf)
145+
private val maxResultSize = conf.get(MAX_RESULT_SIZE)
146146

147147
// Maintains the list of running tasks.
148148
private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]

core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import java.util.regex.PatternSyntaxException
2323
import scala.util.matching.Regex
2424

2525
import org.apache.spark.network.util.{ByteUnit, JavaUtils}
26+
import org.apache.spark.util.Utils
2627

2728
private object ConfigHelpers {
2829

@@ -45,7 +46,7 @@ private object ConfigHelpers {
4546
}
4647

4748
def stringToSeq[T](str: String, converter: String => T): Seq[T] = {
48-
str.split(",").map(_.trim()).filter(_.nonEmpty).map(converter)
49+
Utils.stringToSeq(str).map(converter)
4950
}
5051

5152
def seqToString[T](v: Seq[T], stringConverter: T => String): String = {

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -520,4 +520,9 @@ package object config {
520520
.checkValue(v => v > 0, "The threshold should be positive.")
521521
.createWithDefault(10000000)
522522

523+
private[spark] val MAX_RESULT_SIZE = ConfigBuilder("spark.driver.maxResultSize")
524+
.doc("Size limit for results.")
525+
.bytesConf(ByteUnit.BYTE)
526+
.createWithDefaultString("1g")
527+
523528
}

core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -64,8 +64,7 @@ private[spark] class TaskSetManager(
6464
val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75)
6565
val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5)
6666

67-
// Limit of bytes for total size of results (default is 1GB)
68-
val maxResultSize = Utils.getMaxResultSize(conf)
67+
val maxResultSize = conf.get(config.MAX_RESULT_SIZE)
6968

7069
val speculationEnabled = conf.getBoolean("spark.speculation", false)
7170

0 commit comments

Comments
 (0)