diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 7fcb7830e7b0b4bab0313ae46fc913b166ff9f83..87ab099267b2f5e0f046acdddc9c8e3fe8ec5bc8 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -121,6 +121,7 @@ trait Logging {
     if (usingLog4j12) {
       val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
       if (!log4j12Initialized) {
+        // scalastyle:off println
         if (Utils.isInInterpreter) {
           val replDefaultLogProps = "org/apache/spark/log4j-defaults-repl.properties"
           Option(Utils.getSparkClassLoader.getResource(replDefaultLogProps)) match {
@@ -141,6 +142,7 @@ trait Logging {
               System.err.println(s"Spark was unable to load $defaultLogProps")
           }
         }
+        // scalastyle:on println
       }
     }
     Logging.initialized = true
diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
index 1a5f2bca26c2b32c851b286a2e177d181a111183..b7e72d4d0ed0b750b60229b52b3a8bc1b82812b5 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
@@ -95,7 +95,9 @@ private[spark] class RBackend {
 private[spark] object RBackend extends Logging {
   def main(args: Array[String]): Unit = {
     if (args.length < 1) {
+      // scalastyle:off println
       System.err.println("Usage: RBackend <tempFilePath>")
+      // scalastyle:on println
       System.exit(-1)
     }
     val sparkRBackend = new RBackend()
diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
index 524676544d6f5f3bdec80142b714258397b5d3e0..ff1702f7dea48942f0a80417a64d45355d79eef0 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
@@ -161,7 +161,9 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
               dataOut.write(elem.asInstanceOf[Array[Byte]])
             } else if (deserializer == SerializationFormats.STRING) {
               // write string(for StringRRDD)
+              // scalastyle:off println
               printOut.println(elem)
+              // scalastyle:on println
             }
           }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index 71f7e2129116f8583440208d8fbbb3322b355af2..f03875a3e8c89826e9ebb94a8eee0d57e16d6654 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -118,26 +118,26 @@ private class ClientEndpoint(
   def pollAndReportStatus(driverId: String) {
     // Since ClientEndpoint is the only RpcEndpoint in the process, blocking the event loop thread
     // is fine.
-    println("... waiting before polling master for driver state")
+    logInfo("... waiting before polling master for driver state")
     Thread.sleep(5000)
-    println("... polling master for driver state")
+    logInfo("... polling master for driver state")
     val statusResponse =
       activeMasterEndpoint.askWithRetry[DriverStatusResponse](RequestDriverStatus(driverId))
     statusResponse.found match {
       case false =>
-        println(s"ERROR: Cluster master did not recognize $driverId")
+        logError(s"ERROR: Cluster master did not recognize $driverId")
         System.exit(-1)
       case true =>
-        println(s"State of $driverId is ${statusResponse.state.get}")
+        logInfo(s"State of $driverId is ${statusResponse.state.get}")
         // Worker node, if present
         (statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match {
           case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) =>
-            println(s"Driver running on $hostPort ($id)")
+            logInfo(s"Driver running on $hostPort ($id)")
           case _ =>
         }
         // Exception, if present
         statusResponse.exception.map { e =>
-          println(s"Exception from cluster was: $e")
+          logError(s"Exception from cluster was: $e")
           e.printStackTrace()
           System.exit(-1)
         }
@@ -148,7 +148,7 @@ private class ClientEndpoint(
   override def receive: PartialFunction[Any, Unit] = {
 
     case SubmitDriverResponse(master, success, driverId, message) =>
-      println(message)
+      logInfo(message)
       if (success) {
         activeMasterEndpoint = master
         pollAndReportStatus(driverId.get)
@@ -158,7 +158,7 @@ private class ClientEndpoint(
 
 
     case KillDriverResponse(master, driverId, success, message) =>
-      println(message)
+      logInfo(message)
       if (success) {
         activeMasterEndpoint = master
         pollAndReportStatus(driverId)
@@ -169,13 +169,13 @@ private class ClientEndpoint(
 
   override def onDisconnected(remoteAddress: RpcAddress): Unit = {
     if (!lostMasters.contains(remoteAddress)) {
-      println(s"Error connecting to master $remoteAddress.")
+      logError(s"Error connecting to master $remoteAddress.")
       lostMasters += remoteAddress
       // Note that this heuristic does not account for the fact that a Master can recover within
       // the lifetime of this client. Thus, once a Master is lost it is lost to us forever. This
       // is not currently a concern, however, because this client does not retry submissions.
       if (lostMasters.size >= masterEndpoints.size) {
-        println("No master is available, exiting.")
+        logError("No master is available, exiting.")
         System.exit(-1)
       }
     }
@@ -183,18 +183,18 @@ private class ClientEndpoint(
 
   override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = {
     if (!lostMasters.contains(remoteAddress)) {
-      println(s"Error connecting to master ($remoteAddress).")
-      println(s"Cause was: $cause")
+      logError(s"Error connecting to master ($remoteAddress).")
+      logError(s"Cause was: $cause")
       lostMasters += remoteAddress
       if (lostMasters.size >= masterEndpoints.size) {
-        println("No master is available, exiting.")
+        logError("No master is available, exiting.")
         System.exit(-1)
       }
     }
   }
 
   override def onError(cause: Throwable): Unit = {
-    println(s"Error processing messages, exiting.")
+    logError(s"Error processing messages, exiting.")
     cause.printStackTrace()
     System.exit(-1)
   }
@@ -209,10 +209,12 @@ private class ClientEndpoint(
  */
 object Client {
   def main(args: Array[String]) {
+    // scalastyle:off println
     if (!sys.props.contains("SPARK_SUBMIT")) {
       println("WARNING: This client is deprecated and will be removed in a future version of Spark")
       println("Use ./bin/spark-submit with \"--master spark://host:port\"")
     }
+    // scalastyle:on println
 
     val conf = new SparkConf()
     val driverArgs = new ClientArguments(args)
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 42d3296062e6d9170ed58b90f4070e4c6222b79c..72cc330a398dabcb648e7f51e94d6ebf705438e1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -72,9 +72,11 @@ private[deploy] class ClientArguments(args: Array[String]) {
       cmd = "launch"
 
       if (!ClientArguments.isValidJarUrl(_jarUrl)) {
+        // scalastyle:off println
         println(s"Jar url '${_jarUrl}' is not in valid format.")
         println(s"Must be a jar file path in URL format " +
           "(e.g. hdfs://host:port/XX.jar, file:///XX.jar)")
+        // scalastyle:on println
         printUsageAndExit(-1)
       }
 
@@ -110,7 +112,9 @@ private[deploy] class ClientArguments(args: Array[String]) {
       |                                  (default: $DEFAULT_SUPERVISE)
       |   -v, --verbose                  Print more debugging output
      """.stripMargin
+    // scalastyle:off println
     System.err.println(usage)
+    // scalastyle:on println
     System.exit(exitCode)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
index e99779f299785a045b6a5ea3df879c395f1642c5..4165740312e0330264a8de570011288a373e3f03 100644
--- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala
@@ -85,7 +85,9 @@ object RRunner {
       }
       System.exit(returnCode)
     } else {
+      // scalastyle:off println
       System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds")
+      // scalastyle:on println
       System.exit(-1)
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index b1d6ec209d62ba19b24ccc5d2e65986bb9d586a8..4cec9017b8adbbcedd4d55b9715eeb2d1fbdf4b6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -82,6 +82,7 @@ object SparkSubmit {
 
   private val CLASS_NOT_FOUND_EXIT_STATUS = 101
 
+  // scalastyle:off println
   // Exposed for testing
   private[spark] var exitFn: Int => Unit = (exitCode: Int) => System.exit(exitCode)
   private[spark] var printStream: PrintStream = System.err
@@ -102,11 +103,14 @@ object SparkSubmit {
     printStream.println("Type --help for more information.")
     exitFn(0)
   }
+  // scalastyle:on println
 
   def main(args: Array[String]): Unit = {
     val appArgs = new SparkSubmitArguments(args)
     if (appArgs.verbose) {
+      // scalastyle:off println
       printStream.println(appArgs)
+      // scalastyle:on println
     }
     appArgs.action match {
       case SparkSubmitAction.SUBMIT => submit(appArgs)
@@ -160,7 +164,9 @@ object SparkSubmit {
             // makes the message printed to the output by the JVM not very helpful. Instead,
             // detect exceptions with empty stack traces here, and treat them differently.
             if (e.getStackTrace().length == 0) {
+              // scalastyle:off println
               printStream.println(s"ERROR: ${e.getClass().getName()}: ${e.getMessage()}")
+              // scalastyle:on println
               exitFn(1)
             } else {
               throw e
@@ -178,7 +184,9 @@ object SparkSubmit {
      // to use the legacy gateway if the master endpoint turns out to be not a REST server.
     if (args.isStandaloneCluster && args.useRest) {
       try {
+        // scalastyle:off println
         printStream.println("Running Spark using the REST application submission protocol.")
+        // scalastyle:on println
         doRunMain()
       } catch {
         // Fail over to use the legacy submission gateway
@@ -558,6 +566,7 @@ object SparkSubmit {
       sysProps: Map[String, String],
       childMainClass: String,
       verbose: Boolean): Unit = {
+    // scalastyle:off println
     if (verbose) {
       printStream.println(s"Main class:\n$childMainClass")
       printStream.println(s"Arguments:\n${childArgs.mkString("\n")}")
@@ -565,6 +574,7 @@ object SparkSubmit {
       printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}")
       printStream.println("\n")
     }
+    // scalastyle:on println
 
     val loader =
       if (sysProps.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) {
@@ -592,8 +602,10 @@ object SparkSubmit {
       case e: ClassNotFoundException =>
         e.printStackTrace(printStream)
         if (childMainClass.contains("thriftserver")) {
+          // scalastyle:off println
           printStream.println(s"Failed to load main class $childMainClass.")
           printStream.println("You need to build Spark with -Phive and -Phive-thriftserver.")
+          // scalastyle:on println
         }
         System.exit(CLASS_NOT_FOUND_EXIT_STATUS)
     }
@@ -766,7 +778,9 @@ private[spark] object SparkSubmitUtils {
         brr.setRoot(repo)
         brr.setName(s"repo-${i + 1}")
         cr.add(brr)
+        // scalastyle:off println
         printStream.println(s"$repo added as a remote repository with the name: ${brr.getName}")
+        // scalastyle:on println
       }
     }
 
@@ -829,7 +843,9 @@ private[spark] object SparkSubmitUtils {
       val ri = ModuleRevisionId.newInstance(mvn.groupId, mvn.artifactId, mvn.version)
       val dd = new DefaultDependencyDescriptor(ri, false, false)
       dd.addDependencyConfiguration(ivyConfName, ivyConfName)
+      // scalastyle:off println
       printStream.println(s"${dd.getDependencyId} added as a dependency")
+      // scalastyle:on println
       md.addDependency(dd)
     }
   }
@@ -896,9 +912,11 @@ private[spark] object SparkSubmitUtils {
             ivySettings.setDefaultCache(new File(alternateIvyCache, "cache"))
             new File(alternateIvyCache, "jars")
           }
+        // scalastyle:off println
         printStream.println(
           s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}")
         printStream.println(s"The jars for the packages stored in: $packagesDirectory")
+        // scalastyle:on println
         // create a pattern matcher
         ivySettings.addMatcher(new GlobPatternMatcher)
         // create the dependency resolvers
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index 6e3c0b21b33c2cde9e0167e260c1954b1ec732bc..ebb39c354dff190790589af1ae160bbed6bb71ea 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -79,6 +79,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
   /** Default properties present in the currently defined defaults file. */
   lazy val defaultSparkProperties: HashMap[String, String] = {
     val defaultProperties = new HashMap[String, String]()
+    // scalastyle:off println
     if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile")
     Option(propertiesFile).foreach { filename =>
       Utils.getPropertiesFromFile(filename).foreach { case (k, v) =>
@@ -86,6 +87,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
         if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v")
       }
     }
+    // scalastyle:on println
     defaultProperties
   }
 
@@ -452,6 +454,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
   }
 
   private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = {
+    // scalastyle:off println
     val outStream = SparkSubmit.printStream
     if (unknownParam != null) {
       outStream.println("Unknown/unsupported param " + unknownParam)
@@ -541,6 +544,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
       outStream.println("CLI options:")
       outStream.println(getSqlShellOptions())
     }
+    // scalastyle:on println
 
     SparkSubmit.exitFn(exitCode)
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
index c5ac45c6730d3f45f41b200d88ee62f96d16c0c1..a98b1fa8f83a1707e69b16e4c3388dec204988b5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
@@ -19,7 +19,9 @@ package org.apache.spark.deploy.client
 
 private[spark] object TestExecutor {
   def main(args: Array[String]) {
+    // scalastyle:off println
     println("Hello world!")
+    // scalastyle:on println
     while (true) {
       Thread.sleep(1000)
     }
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
index 4692d22651c939349fab0b73e3596371346b8729..18265df9faa2cc7cb1d96d51e2652f050f64717e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
@@ -56,6 +56,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin
    Utils.loadDefaultSparkProperties(conf, propertiesFile)
 
   private def printUsageAndExit(exitCode: Int) {
+    // scalastyle:off println
     System.err.println(
       """
       |Usage: HistoryServer [options]
@@ -84,6 +85,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin
       |  spark.history.fs.updateInterval    How often to reload log data from storage
       |                                     (in seconds, default: 10)
       |""".stripMargin)
+    // scalastyle:on println
     System.exit(exitCode)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 435b9b12f83b82798fbfd4b46f55557409ca5a5f..44cefbc77f08e060a2ef8b65daa6a0e6220fbf2e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -85,6 +85,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
    * Print usage and exit JVM with the given exit code.
    */
   private def printUsageAndExit(exitCode: Int) {
+    // scalastyle:off println
     System.err.println(
       "Usage: Master [options]\n" +
       "\n" +
@@ -95,6 +96,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
       "  --webui-port PORT      Port for web UI (default: 8080)\n" +
       "  --properties-file FILE Path to a custom Spark properties file.\n" +
       "                         Default is conf/spark-defaults.conf.")
+    // scalastyle:on println
     System.exit(exitCode)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
index 894cb78d8591a7492e020999497c60ea61a167ab..5accaf78d0a518058ca0d700d1469e0221d7e6dd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
@@ -54,7 +54,9 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
 
     case ("--master" | "-m") :: value :: tail =>
       if (!value.startsWith("mesos://")) {
+        // scalastyle:off println
         System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)")
+        // scalastyle:on println
         System.exit(1)
       }
       masterUrl = value.stripPrefix("mesos://")
@@ -73,7 +75,9 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
 
     case Nil => {
       if (masterUrl == null) {
+        // scalastyle:off println
         System.err.println("--master is required")
+        // scalastyle:on println
         printUsageAndExit(1)
       }
     }
@@ -83,6 +87,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
   }
 
   private def printUsageAndExit(exitCode: Int): Unit = {
+    // scalastyle:off println
     System.err.println(
       "Usage: MesosClusterDispatcher [options]\n" +
         "\n" +
@@ -96,6 +101,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
         "                          Zookeeper for persistence\n" +
         "  --properties-file FILE  Path to a custom Spark properties file.\n" +
         "                          Default is conf/spark-defaults.conf.")
+    // scalastyle:on println
     System.exit(exitCode)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index d1a12b01e78f7867af604861abcac8b332404e37..2d6be3042c9053aaa10a2ca55cc05cf6ce1afa4a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -60,7 +60,9 @@ object DriverWrapper {
         rpcEnv.shutdown()
 
       case _ =>
+        // scalastyle:off println
         System.err.println("Usage: DriverWrapper <workerUrl> <userJar> <driverMainClass> [options]")
+        // scalastyle:on println
         System.exit(-1)
     }
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 1d2ecab517613a7bc23be78536e5e164b8744ac0..e89d076802215321510f2f1a73872e0873114321 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -121,6 +121,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
    * Print usage and exit JVM with the given exit code.
    */
   def printUsageAndExit(exitCode: Int) {
+    // scalastyle:off println
     System.err.println(
       "Usage: Worker [options] <master>\n" +
       "\n" +
@@ -136,6 +137,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
       "  --webui-port PORT        Port for web UI (default: 8081)\n" +
       "  --properties-file FILE   Path to a custom Spark properties file.\n" +
       "                           Default is conf/spark-defaults.conf.")
+    // scalastyle:on println
     System.exit(exitCode)
   }
 
@@ -160,7 +162,9 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
     } catch {
       case e: Exception => {
         totalMb = 2*1024
+        // scalastyle:off println
         System.out.println("Failed to get total physical memory. Using " + totalMb + " MB")
+        // scalastyle:on println
       }
     }
     // Leave out 1 GB for the operating system, but don't return a negative memory size
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 34d4cfdca773251deecf6ba717c21779813e2c55..fcd76ec52742a2809d0044a47970f758321d5ceb 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -235,7 +235,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
           argv = tail
         case Nil =>
         case tail =>
+          // scalastyle:off println
           System.err.println(s"Unrecognized options: ${tail.mkString(" ")}")
+          // scalastyle:on println
           printUsageAndExit()
       }
     }
@@ -249,6 +251,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
   }
 
   private def printUsageAndExit() = {
+    // scalastyle:off println
     System.err.println(
       """
       |"Usage: CoarseGrainedExecutorBackend [options]
@@ -262,6 +265,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
       |   --worker-url <workerUrl>
       |   --user-class-path <url>
       |""".stripMargin)
+    // scalastyle:on println
     System.exit(1)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
index c219d21fbefa90b19304de97f5559b2ba8c2e754..532850dd5771650973e7feea978b5c52e01070a3 100644
--- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
+++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala
@@ -21,6 +21,8 @@ import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.{BytesWritable, LongWritable}
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext}
+
+import org.apache.spark.Logging
 import org.apache.spark.deploy.SparkHadoopUtil
 
 /**
@@ -39,7 +41,8 @@ private[spark] object FixedLengthBinaryInputFormat {
 }
 
 private[spark] class FixedLengthBinaryInputFormat
-  extends FileInputFormat[LongWritable, BytesWritable] {
+  extends FileInputFormat[LongWritable, BytesWritable]
+  with Logging {
 
   private var recordLength = -1
 
@@ -51,7 +54,7 @@ private[spark] class FixedLengthBinaryInputFormat
       recordLength = FixedLengthBinaryInputFormat.getRecordLength(context)
     }
     if (recordLength <= 0) {
-      println("record length is less than 0, file cannot be split")
+      logDebug("record length is less than 0, file cannot be split")
       false
     } else {
       true
diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala
index 67a376102994c06a918c7cb10af074a6fb8350dd..79cb0640c8672fcfabc0aa2245a570e631513f79 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala
@@ -57,16 +57,6 @@ private[nio] class BlockMessage() {
   }
 
   def set(buffer: ByteBuffer) {
-    /*
-    println()
-    println("BlockMessage: ")
-    while(buffer.remaining > 0) {
-      print(buffer.get())
-    }
-    buffer.rewind()
-    println()
-    println()
-    */
     typ = buffer.getInt()
     val idLength = buffer.getInt()
     val idBuilder = new StringBuilder(idLength)
@@ -138,18 +128,6 @@ private[nio] class BlockMessage() {
       buffers += data
     }
 
-    /*
-    println()
-    println("BlockMessage: ")
-    buffers.foreach(b => {
-      while(b.remaining > 0) {
-        print(b.get())
-      }
-      b.rewind()
-    })
-    println()
-    println()
-    */
     Message.createBufferMessage(buffers)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
index 7d0806f0c2580474aa85477629c6ab2bc8f94d87..f1c9ea8b64ca3d6c35196486131736a47f652ae7 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
@@ -43,16 +43,6 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage])
     val newBlockMessages = new ArrayBuffer[BlockMessage]()
     val buffer = bufferMessage.buffers(0)
     buffer.clear()
-    /*
-    println()
-    println("BlockMessageArray: ")
-    while(buffer.remaining > 0) {
-      print(buffer.get())
-    }
-    buffer.rewind()
-    println()
-    println()
-    */
     while (buffer.remaining() > 0) {
       val size = buffer.getInt()
       logDebug("Creating block message of size " + size + " bytes")
@@ -86,23 +76,11 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage])
 
     logDebug("Buffer list:")
     buffers.foreach((x: ByteBuffer) => logDebug("" + x))
-    /*
-    println()
-    println("BlockMessageArray: ")
-    buffers.foreach(b => {
-      while(b.remaining > 0) {
-        print(b.get())
-      }
-      b.rewind()
-    })
-    println()
-    println()
-    */
     Message.createBufferMessage(buffers)
   }
 }
 
-private[nio] object BlockMessageArray {
+private[nio] object BlockMessageArray extends Logging {
 
   def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = {
     val newBlockMessageArray = new BlockMessageArray()
@@ -123,10 +101,10 @@ private[nio] object BlockMessageArray {
         }
       }
     val blockMessageArray = new BlockMessageArray(blockMessages)
-    println("Block message array created")
+    logDebug("Block message array created")
 
     val bufferMessage = blockMessageArray.toBufferMessage
-    println("Converted to buffer message")
+    logDebug("Converted to buffer message")
 
     val totalSize = bufferMessage.size
     val newBuffer = ByteBuffer.allocate(totalSize)
@@ -138,10 +116,11 @@ private[nio] object BlockMessageArray {
     })
     newBuffer.flip
     val newBufferMessage = Message.createBufferMessage(newBuffer)
-    println("Copied to new buffer message, size = " + newBufferMessage.size)
+    logDebug("Copied to new buffer message, size = " + newBufferMessage.size)
 
     val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage)
-    println("Converted back to block message array")
+    logDebug("Converted back to block message array")
+    // scalastyle:off println
     newBlockMessageArray.foreach(blockMessage => {
       blockMessage.getType match {
         case BlockMessage.TYPE_PUT_BLOCK => {
@@ -154,6 +133,7 @@ private[nio] object BlockMessageArray {
         }
       }
     })
+    // scalastyle:on println
   }
 }
 
diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
index c0bca2c4bc9947ef20e04d7156a91881b59bbfcd..91439187903817fcef7d53a020761a182d9382d9 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
@@ -1016,7 +1016,9 @@ private[spark] object ConnectionManager {
     val conf = new SparkConf
     val manager = new ConnectionManager(9999, conf, new SecurityManager(conf))
     manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
+      // scalastyle:off println
       println("Received [" + msg + "] from [" + id + "]")
+      // scalastyle:on println
       None
     })
 
@@ -1033,6 +1035,7 @@ private[spark] object ConnectionManager {
     System.gc()
   }
 
+  // scalastyle:off println
   def testSequentialSending(manager: ConnectionManager) {
     println("--------------------------")
     println("Sequential Sending")
@@ -1150,4 +1153,5 @@ private[spark] object ConnectionManager {
       println()
     }
   }
+  // scalastyle:on println
 }
diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index dc60d489276240b20a840f6de650b5a793d9f933..defdabf95ac4b79f781271e9e9a1fd251afa07c6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -123,7 +123,9 @@ private[spark] class PipedRDD[T: ClassTag](
     new Thread("stderr reader for " + command) {
       override def run() {
         for (line <- Source.fromInputStream(proc.getErrorStream).getLines) {
+          // scalastyle:off println
           System.err.println(line)
+          // scalastyle:on println
         }
       }
     }.start()
@@ -133,6 +135,7 @@ private[spark] class PipedRDD[T: ClassTag](
       override def run() {
         val out = new PrintWriter(proc.getOutputStream)
 
+        // scalastyle:off println
         // input the pipe context firstly
         if (printPipeContext != null) {
           printPipeContext(out.println(_))
@@ -144,6 +147,7 @@ private[spark] class PipedRDD[T: ClassTag](
             out.println(elem)
           }
         }
+        // scalastyle:on println
         out.close()
       }
     }.start()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 529a5b2bf1a0d616606fb19dde4f4884523b1f18..62b05033a928193a0149ababb9efc0f14e8abec7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -140,7 +140,9 @@ private[spark] class EventLoggingListener(
   /** Log the event as JSON. */
   private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) {
     val eventJson = JsonProtocol.sparkEventToJson(event)
+    // scalastyle:off println
     writer.foreach(_.println(compact(render(eventJson))))
+    // scalastyle:on println
     if (flushLogger) {
       writer.foreach(_.flush())
       hadoopDataStream.foreach(hadoopFlushMethod.invoke(_))
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index e55b76c36cc5f229f666172023ea23771172d84d..f96eb8ca0ae00fed6180ef805e2a47fb3b55f314 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -125,7 +125,9 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener
       val date = new Date(System.currentTimeMillis())
       writeInfo = dateFormat.get.format(date) + ": " + info
     }
+    // scalastyle:off println
     jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo))
+    // scalastyle:on println
   }
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index f413c1d37fbb6be0cd7d6f3e5344fd112b25a6f1..c8356467fab872abcedbffbd1b82db798fda9825 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -68,7 +68,9 @@ private[spark] object JettyUtils extends Logging {
             response.setStatus(HttpServletResponse.SC_OK)
             val result = servletParams.responder(request)
             response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
+            // scalastyle:off println
             response.getWriter.println(servletParams.extractFn(result))
+            // scalastyle:on println
           } else {
             response.setStatus(HttpServletResponse.SC_UNAUTHORIZED)
             response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index ba03acdb38cc5da23dcfc9ccedcf9583a721749e..5a8c2914314c219aad938e50789c43eba291d008 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -38,9 +38,11 @@ private[spark] object UIWorkloadGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 3) {
+      // scalastyle:off println
       println(
-        "usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator " +
+        "Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator " +
           "[master] [FIFO|FAIR] [#job set (4 jobs per set)]")
+      // scalastyle:on println
       System.exit(1)
     }
 
@@ -96,6 +98,7 @@ private[spark] object UIWorkloadGenerator {
       for ((desc, job) <- jobs) {
         new Thread {
           override def run() {
+            // scalastyle:off println
             try {
               setProperties(desc)
               job()
@@ -106,6 +109,7 @@ private[spark] object UIWorkloadGenerator {
             } finally {
               barrier.release()
             }
+            // scalastyle:on println
           }
         }.start
         Thread.sleep(INTER_JOB_WAIT_MS)
diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala
index 1bab707235b89f0a8f8bf7f6d64d47c04baa69b9..950b69f7db6415f82c8b40ba7fe8acd21c0b248c 100644
--- a/core/src/main/scala/org/apache/spark/util/Distribution.scala
+++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala
@@ -52,9 +52,11 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va
   }
 
   def showQuantiles(out: PrintStream = System.out): Unit = {
+    // scalastyle:off println
     out.println("min\t25%\t50%\t75%\tmax")
     getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")}
     out.println
+    // scalastyle:on println
   }
 
   def statCounter: StatCounter = StatCounter(data.slice(startIdx, endIdx))
@@ -64,8 +66,10 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va
    * @param out
    */
   def summary(out: PrintStream = System.out) {
+    // scalastyle:off println
     out.println(statCounter)
     showQuantiles(out)
+    // scalastyle:on println
   }
 }
 
@@ -80,8 +84,10 @@ private[spark] object Distribution {
   }
 
   def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) {
+    // scalastyle:off println
     out.println("min\t25%\t50%\t75%\tmax")
     quantiles.foreach{q => out.print(q + "\t")}
     out.println
+    // scalastyle:on println
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
index c4a7b4441c85cf5c8914fc6e1a46b7c51f4ea261..85fb923cd9bc74d763ca15cac3bb1c9a681db9f1 100644
--- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
@@ -70,12 +70,14 @@ private[spark] object XORShiftRandom {
    * @param args takes one argument - the number of random numbers to generate
    */
   def main(args: Array[String]): Unit = {
+    // scalastyle:off println
     if (args.length != 1) {
       println("Benchmark of XORShiftRandom vis-a-vis java.util.Random")
       println("Usage: XORShiftRandom number_of_random_numbers_to_generate")
       System.exit(1)
     }
     println(benchmark(args(0).toInt))
+    // scalastyle:on println
   }
 
   /**
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 9c191ed52206da4922a33f257d96a0b142758a6c..2300bcff4f1184f83d44acca5790291720e547d5 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -107,7 +107,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
     sc = new SparkContext(clusterUrl, "test")
     val accum = sc.accumulator(0)
     val thrown = intercept[SparkException] {
+      // scalastyle:off println
       sc.parallelize(1 to 10, 10).foreach(x => println(x / 0))
+      // scalastyle:on println
     }
     assert(thrown.getClass === classOf[SparkException])
     assert(thrown.getMessage.contains("failed 4 times"))
diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala
index a8c8c6f73fb5adca12a57fd2f67bb6b62754d4a2..b099cd3fb796554695e0b4c8ff793a6e68c30f06 100644
--- a/core/src/test/scala/org/apache/spark/FailureSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala
@@ -130,7 +130,9 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext {
 
     // Non-serializable closure in foreach function
     val thrown2 = intercept[SparkException] {
+      // scalastyle:off println
       sc.parallelize(1 to 10, 2).foreach(x => println(a))
+      // scalastyle:on println
     }
     assert(thrown2.getClass === classOf[SparkException])
     assert(thrown2.getMessage.contains("NotSerializableException") ||
diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
index 6e65b0a8f6c76c131b963356c9e507de2c8e8ec6..876418aa1302941c597161fcad8cd319caf2a4d5 100644
--- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -51,7 +51,9 @@ class FileServerSuite extends SparkFunSuite with LocalSparkContext {
 
     val textFile = new File(testTempDir, "FileServerSuite.txt")
     val pw = new PrintWriter(textFile)
+    // scalastyle:off println
     pw.println("100")
+    // scalastyle:on println
     pw.close()
 
     val jarFile = new File(testTempDir, "test.jar")
diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
index 6580139df6c60bc3788b7e90394c4a79596f9721..48509f0759a3bdbefacac82709edf2ff3d671d53 100644
--- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
@@ -36,7 +36,7 @@ object ThreadingSuiteState {
   }
 }
 
-class ThreadingSuite extends SparkFunSuite with LocalSparkContext {
+class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
 
   test("accessing SparkContext form a different thread") {
     sc = new SparkContext("local", "test")
@@ -130,8 +130,6 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext {
               Thread.sleep(100)
             }
             if (running.get() != 4) {
-              println("Waited 1 second without seeing runningThreads = 4 (it was " +
-                running.get() + "); failing test")
               ThreadingSuiteState.failed.set(true)
             }
             number
@@ -143,6 +141,8 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext {
     }
     sem.acquire(2)
     if (ThreadingSuiteState.failed.get()) {
+      logError("Waited 1 second without seeing runningThreads = 4 (it was " +
+                ThreadingSuiteState.runningThreads.get() + "); failing test")
       fail("One or more threads didn't see runningThreads = 4")
     }
   }
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 2e05dec99b6bf281e897bba75d188177f67c5577..1b64c329b5d4bc187eeea4d6ea0d0d90976110c9 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -51,9 +51,11 @@ class SparkSubmitSuite
   /** Simple PrintStream that reads data into a buffer */
   private class BufferPrintStream extends PrintStream(noOpOutputStream) {
     var lineBuffer = ArrayBuffer[String]()
+    // scalastyle:off println
     override def println(line: String) {
       lineBuffer += line
     }
+    // scalastyle:on println
   }
 
   /** Returns true if the script exits and the given search string is printed. */
@@ -81,6 +83,7 @@ class SparkSubmitSuite
     }
   }
 
+  // scalastyle:off println
   test("prints usage on empty input") {
     testPrematureExit(Array[String](), "Usage: spark-submit")
   }
@@ -491,6 +494,7 @@ class SparkSubmitSuite
       appArgs.executorMemory should be ("2.3g")
     }
   }
+  // scalastyle:on println
 
   // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
   private def runSparkSubmit(args: Seq[String]): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
index c9b435a9228d3977c5fdb05fccda2ab18709c292..01ece1a10f46db3b0162d46f4b69897c1034a0c9 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala
@@ -41,9 +41,11 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll {
   /** Simple PrintStream that reads data into a buffer */
   private class BufferPrintStream extends PrintStream(noOpOutputStream) {
     var lineBuffer = ArrayBuffer[String]()
+    // scalastyle:off println
     override def println(line: String) {
       lineBuffer += line
     }
+    // scalastyle:on println
   }
 
   override def beforeAll() {
diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
index 63947df3d43a25adf90772a6024d7c596e77be68..8a199459c1ddf30fa8e4499a4f2a920b8d5565f2 100644
--- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala
@@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.hadoop.io.Text
 
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite}
 import org.apache.spark.util.Utils
 import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec}
 
@@ -36,7 +36,7 @@ import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, Gzi
  * [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary
  * directory is created as fake input. Temporal storage would be deleted in the end.
  */
-class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAll {
+class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
   private var sc: SparkContext = _
   private var factory: CompressionCodecFactory = _
 
@@ -85,7 +85,7 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl
    */
   test("Correctness of WholeTextFileRecordReader.") {
     val dir = Utils.createTempDir()
-    println(s"Local disk address is ${dir.toString}.")
+    logInfo(s"Local disk address is ${dir.toString}.")
 
     WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) =>
       createNativeFile(dir, filename, contents, false)
@@ -109,7 +109,7 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl
 
   test("Correctness of WholeTextFileRecordReader with GzipCodec.") {
     val dir = Utils.createTempDir()
-    println(s"Local disk address is ${dir.toString}.")
+    logInfo(s"Local disk address is ${dir.toString}.")
 
     WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) =>
       createNativeFile(dir, filename, contents, true)
diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
index 9e4d34fb7d3820ec3d95478082fd1df6319efcce..d3218a548efc7af176535a822641f865529eefe1 100644
--- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala
@@ -60,7 +60,9 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext
     tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt")
     val pw = new PrintWriter(new FileWriter(tmpFile))
     for (x <- 1 to numRecords) {
+      // scalastyle:off println
       pw.println(RandomUtils.nextInt(0, numBuckets))
+      // scalastyle:on println
     }
     pw.close()
 
diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
index ff3fa95ec32ae0771d66ed9a9da5dfb29f3368d1..4e3defb43a021ee8fa8ca966d7d2ab895acc055a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
@@ -52,8 +52,10 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter {
     val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None,
       125L, "Mickey", None)
     val applicationEnd = SparkListenerApplicationEnd(1000L)
+    // scalastyle:off println
     writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart))))
     writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd))))
+    // scalastyle:on println
     writer.close()
 
     val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath)
diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala
index 1053c6caf771858b7aa2647621b5e34325063cac..480722a5ac182054cbea0204748656df154f8e49 100644
--- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala
@@ -375,6 +375,7 @@ class TestCreateNullValue {
     // parameters of the closure constructor. This allows us to test whether
     // null values are created correctly for each type.
     val nestedClosure = () => {
+      // scalastyle:off println
       if (s.toString == "123") { // Don't really output them to avoid noisy
         println(bo)
         println(c)
@@ -389,6 +390,7 @@ class TestCreateNullValue {
       val closure = () => {
         println(getX)
       }
+      // scalastyle:on println
       ClosureCleaner.clean(closure)
     }
     nestedClosure()
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 251a797dc28a270da2e46023a0704b09fc965586..c7638507c88c60677900a1b1ac429cbca328fd45 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -684,7 +684,9 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
     val buffer = new CircularBuffer(25)
     val stream = new java.io.PrintStream(buffer, true, "UTF-8")
 
+    // scalastyle:off println
     stream.println("test circular test circular test circular test circular test circular")
+    // scalastyle:on println
     assert(buffer.toString === "t circular test circular\n")
   }
 }
diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala
index 5a5919fca24693ca0e50814c62704401d398affc..4f382414a8dd7b8680ec6a851445852931c6bdc1 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala
@@ -103,7 +103,9 @@ private object SizeTrackerSuite {
    */
   def main(args: Array[String]): Unit = {
     if (args.size < 1) {
+      // scalastyle:off println
       println("Usage: SizeTrackerSuite [num elements]")
+      // scalastyle:on println
       System.exit(1)
     }
     val numElements = args(0).toInt
@@ -180,11 +182,13 @@ private object SizeTrackerSuite {
       baseTimes: Seq[Long],
       sampledTimes: Seq[Long],
       unsampledTimes: Seq[Long]): Unit = {
+    // scalastyle:off println
     println(s"Average times for $testName (ms):")
     println("  Base - " + averageTime(baseTimes))
     println("  SizeTracker (sampled) - " + averageTime(sampledTimes))
     println("  SizeEstimator (unsampled) - " + averageTime(unsampledTimes))
     println()
+    // scalastyle:on println
   }
 
   def time(f: => Unit): Long = {
diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
index b2f5d9009ee5d71861840d0f546e0c39d3ef0682..fefa5165db19714b42a176d33bb65ccf7c43f6f1 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala
@@ -20,10 +20,10 @@ package org.apache.spark.util.collection
 import java.lang.{Float => JFloat, Integer => JInteger}
 import java.util.{Arrays, Comparator}
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.util.random.XORShiftRandom
 
-class SorterSuite extends SparkFunSuite {
+class SorterSuite extends SparkFunSuite with Logging {
 
   test("equivalent to Arrays.sort") {
     val rand = new XORShiftRandom(123)
@@ -74,7 +74,7 @@ class SorterSuite extends SparkFunSuite {
   /** Runs an experiment several times. */
   def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = {
     if (skip) {
-      println(s"Skipped experiment $name.")
+      logInfo(s"Skipped experiment $name.")
       return
     }
 
@@ -86,11 +86,11 @@ class SorterSuite extends SparkFunSuite {
     while (i < 10) {
       val time = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare))
       next10 += time
-      println(s"$name: Took $time ms")
+      logInfo(s"$name: Took $time ms")
       i += 1
     }
 
-    println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)")
+    logInfo(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)")
   }
 
   /**
diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
index fc03fec9866a6287ffe3c4bca9b598556feb4749..61d91c70e9709bd77084ed387a188409527f0315 100644
--- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
+++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.util.Try
@@ -59,3 +60,4 @@ object SimpleApp {
     }
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
index 0be8e64fbfabd82b798f4a27632619fe3c85d077..9f7ae75d0b47776a79c430f62b8ba36a95b34930 100644
--- a/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
+++ b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.util.Try
@@ -37,3 +38,4 @@ object SimpleApp {
     }
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala
index 24c7f8d6672969edef690c938c0846eac46d302e..2f0b6ef9a5672131803f5a7e759794c8390f6a4c 100644
--- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala
+++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import org.apache.spark.{SparkContext, SparkConf}
@@ -51,3 +52,4 @@ object GraphXApp {
     println("Test succeeded")
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
index 5111bc0adb7728b40bdb7baea8dace2502ceea61..4a980ec071ae4174f49950c81c0eefb38061a6f5 100644
--- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
+++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.collection.mutable.{ListBuffer, Queue}
@@ -55,3 +56,4 @@ object SparkSqlExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala
index 9f8506650147246a52741c578a0cd81cd4c7b9c7..adc25b57d6aa5161c65a4bd8ba5471511e18a295 100644
--- a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala
+++ b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.util.Try
@@ -31,3 +32,4 @@ object SimpleApp {
     }
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
index cc86ef45858c9611e71a013dedad1a106fc7ce6d..69c1154dc095516dfdf1709c2ffc10c10fd81058 100644
--- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
+++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.collection.mutable.{ListBuffer, Queue}
@@ -57,3 +58,4 @@ object SparkSqlExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala
index 58a662bd9b2e841bf62dad1fdf53ecb578181192..d6a074687f4a195e642ff90b09a04cee818935cf 100644
--- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala
+++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package main.scala
 
 import scala.collection.mutable.{ListBuffer, Queue}
@@ -61,3 +62,4 @@ object SparkStreamingExample {
     ssc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index 4c129dbe2d12d5c20cd977e85708130efc034d5a..d812262fd87dc60125ec042c88182017c29de1a5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -52,3 +53,4 @@ object BroadcastTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala
index 023bb3ee2d1081f114bcb7d87a6670565166ca53..36832f51d2ad4393ff709f328187055fe4de5ee5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+ // scalastyle:off println
 package org.apache.spark.examples
 
 import java.nio.ByteBuffer
@@ -140,3 +141,4 @@ object CassandraCQLTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
index ec689474aecb049ff2e3ff4888e6af1bc1904816..96ef3e198e38063b5340c4b0cd420e3332a48d17 100644
--- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.nio.ByteBuffer
@@ -130,6 +131,7 @@ object CassandraTest {
     sc.stop()
   }
 }
+// scalastyle:on println
 
 /*
 create keyspace casDemo;
diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
index 1f12034ce0f57fb193fde8d5ad81d1bbe246c414..d651fe4d6ee75c86df1130796b997df067f8b4f4 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.io.File
@@ -136,3 +137,4 @@ object DFSReadWriteTest {
 
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
index e757283823fc3bcee7e32166c86e2433fe697db3..c42df2b8845d2917e20f184e159b89707ae946a4 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import scala.collection.JavaConversions._
@@ -46,3 +47,4 @@ object DriverSubmissionTest {
     }
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
index 15f6678648b2929a556825c300b77d2471f72082..fa4a3afeecd19d8beaae7d84014ef8796233b9b7 100644
--- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -53,3 +54,4 @@ object GroupByTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
index 95c96111c9b1fd424f06ef5bef27602804290277..244742327a907bf7d30b53467b3832f70edb4154 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.hadoop.hbase.client.HBaseAdmin
@@ -62,3 +63,4 @@ object HBaseTest {
     admin.close()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
index ed2b38e2ca6f885467301b17ad1baa2ce1bfcdae..124dc9af6390f91be9529461bf847eb7609f1689 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.spark._
@@ -41,3 +42,4 @@ object HdfsTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 3d5259463003dbdabecbcd39d5817d6016d1718c..af5f216f28ba405b784b528ecc3d37ad1f08dea2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.commons.math3.linear._
@@ -142,3 +143,4 @@ object LocalALS {
     new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random))
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
index ac2ea35bbd0e02137ad0700f31a4c3d4d6ad383b..9c8aae53cf48d425bdf5c23e83e66089c2fc904f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -73,3 +74,4 @@ object LocalFileLR {
     println("Final w: " + w)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
index 04fc0a033014afd885562bfaf7765d2c3df79778..e7b28d38bdfc625a28a24fe2a21f6088f7f64797 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -119,3 +120,4 @@ object LocalKMeans {
     println("Final centers: " + kPoints)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
index c3fc74a116c0a6e92fb05f2f0a68e08ceee744bb..4f6b092a59ca5a2108254dfdc1470a9d30dd4090 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -77,3 +78,4 @@ object LocalLR {
     println("Final w: " + w)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
index ee6b3ee34aeb2e42e79fb8ce2723262bdce802f0..3d923625f11b6f75fc919e31cf49389e0038bb5a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import scala.math.random
@@ -33,3 +34,4 @@ object LocalPi {
     println("Pi is roughly " + 4 * count / 100000.0)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
index 75c82117cbad2618160c2fb6943c240686fa0b8a..a80de10f4610ad39d8799376b3e58be6cfe36c62 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -83,3 +84,4 @@ object LogQuery {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index 2a5c0c0defe137c93e89ee8d6625f6e3359f5e22..61ce9db914f9f194d7a114a4ec27f6adb53d8092 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.spark.rdd.RDD
@@ -53,3 +54,4 @@ object MultiBroadcastTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
index 5291ab81f459e1f4aa3885954fb5f9ad08b5c095..3b0b00fe4dd0a70f03d9b711672dc9be41bab510 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -67,3 +68,4 @@ object SimpleSkewedGroupByTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
index 017d4e1e5ce13b528e8ffdad73768876680ab48e..719e2176fed3f151de0de0aa4bc4e312d4d0903a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -57,3 +58,4 @@ object SkewedGroupByTest {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 30c4261551837f7e8e54900968cf36709cf7cb68..69799b7c2bb3043e0caa456a75a6b382e57c8e89 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.commons.math3.linear._
@@ -144,3 +145,4 @@ object SparkALS {
     new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random))
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 9099c2fcc90b312054f89d713e3ee5daf587b8f4..505ea5a4c7a8571cab5e312e257964650569a396 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -97,3 +98,4 @@ object SparkHdfsLR {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index b514d9123f5e770268bbf2f7b31f696789ca4d55..c56e1124ad41588274f045d3c4c43b3289e91e58 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import breeze.linalg.{Vector, DenseVector, squaredDistance}
@@ -100,3 +101,4 @@ object SparkKMeans {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
index 1e6b4fb0c75148d5cd18333e1b68c413c29a04a7..d265c227f4ed21b977483e1ebcbc2c84a5a7fa00 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -86,3 +87,4 @@ object SparkLR {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
index bd7894f184c4cf92a4bfee5ef74e62dcdf99ccf1..0fd79660dd1966af6bc02fb6706b28693cc78b13 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import org.apache.spark.SparkContext._
@@ -74,3 +75,4 @@ object SparkPageRank {
     ctx.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index 35b8dd6c29b66a69b44644ae3a12e78605fa0b9a..818d4f2b81f828337af7e054a0ecb28de2c59a4a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import scala.math.random
@@ -37,3 +38,4 @@ object SparkPi {
     spark.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 772cd897f5140f979ff10bbc9f39994370f7af07..95072071ccddb643f7266ceddd5fde8bad1152d0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import scala.util.Random
@@ -70,3 +71,4 @@ object SparkTC {
     spark.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala
index 4393b99e636b6d69251fc7f5c4b38d2acef97e9c..cfbdae02212a51a1ba15931f26c2cec9149be5df 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import java.util.Random
@@ -94,3 +95,4 @@ object SparkTachyonHdfsLR {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala
index 7743f7968b1009e06b0f6ae4151d3cba5aa4b70b..e46ac655beb58a8a9c4619eb19104fa11a084fd2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples
 
 import scala.math.random
@@ -46,3 +47,4 @@ object SparkTachyonPi {
     spark.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
index 409721b01c8fd0e2da451ba6cd0cdc352f51f239..8dd6c9706e7df0a94d50a2db33668c3caf9c4a82 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.graphx
 
 import scala.collection.mutable
@@ -151,3 +152,4 @@ object Analytics extends Logging {
     }
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
index f6f8d9f90c2753f799017bdc4d8e3929d016f4d0..da3ffca1a6f2a74d3d9d720a66fe711000726b81 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.graphx
 
 /**
@@ -42,3 +43,4 @@ object LiveJournalPageRank {
     Analytics.main(args.patch(0, List("pagerank"), 0))
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
index 3ec20d594b784ee52af66d778352f14c35fa8306..46e52aacd90bb549bec8266881ade0319a0b1e73 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.graphx
 
 import org.apache.spark.SparkContext._
@@ -128,3 +129,4 @@ object SynthBenchmark {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
index 6c0af20461d3b67d5e0ca299e913ebefddd521a0..14b358d46f6ab8d802324f20535e6ad9fc155865 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -110,3 +111,4 @@ object CrossValidatorExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
index 54e407394105631915b13eeb97e4804e98131b82..f28671f7869fcf206512b0951e270f8ae240a2f9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.collection.mutable
@@ -355,3 +356,4 @@ object DecisionTreeExample {
     println(s"  Root mean squared error (RMSE): $RMSE")
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
index 7b8cc21ed8982e8b7cb3f7c1d89f7c27a64a6eab..78f31b4ffe56a2594b614026543612d7890b226f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -181,3 +182,4 @@ private class MyLogisticRegressionModel(
     copyValues(new MyLogisticRegressionModel(uid, weights), extra)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
index 33905277c7341cedac735b01902e0adf18141cd0..f4a15f806ea81014f5929704426dd01f86d7c6f3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.collection.mutable
@@ -236,3 +237,4 @@ object GBTExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
index b54466fd48bc5b747c4e00cdb124f98e7017df6a..b73299fb12d3f092c3764293168d3830c76f0c96 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.collection.mutable
@@ -140,3 +141,4 @@ object LinearRegressionExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
index 3cf193f353fbc1ba37d9a38b3d6ca13aa0a7e167..7682557127b51273c440af198d7401541d5a4035 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.collection.mutable
@@ -157,3 +158,4 @@ object LogisticRegressionExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
index 25f21113bf62293298daff56f35d0fb89d4875e6..cd411397a4b9dc7ac472b9ec499b95bc6eb348fe 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scopt.OptionParser
@@ -178,3 +179,4 @@ object MovieLensALS {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
index 6927eb8f275cf70b4b1361ce098b83cc00313731..bab31f585b0ef136af273d40f843a1f1c8b437da 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO}
@@ -183,3 +184,4 @@ object OneVsRestExample {
     (NANO.toSeconds(t1 - t0), result)
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
index 9f7cad68a45948e329e0053c639f78d66cbe4874..109178f4137b2cafb7042fbc7678280c60ccc59f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.collection.mutable
@@ -244,3 +245,4 @@ object RandomForestExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
index a0561e2573fc987da39063f5926e5988bb954b1e..58d7b67674ff7a17cdefc9a6c1f72fa513d5fd17 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -100,3 +101,4 @@ object SimpleParamsExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
index 1324b066c30c3f1b7ae489c4541536839b24376c..960280137cbf9007f336c0b29ee6cab4d4e63882 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.ml
 
 import scala.beans.BeanInfo
@@ -89,3 +90,4 @@ object SimpleTextClassificationPipeline {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
index a113653810b93ed10f98c986a943227debc2a95f..1a4016f76c2ad62aa4e65e7633abd7abfdd7dafa 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.log4j.{Level, Logger}
@@ -153,3 +154,4 @@ object BinaryClassification {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala
index e49129c4e7844dec584d88750cddfe825cfcab1f..026d4ecc6d10a3e80191a2bce174ab2e95fe7a79 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scopt.OptionParser
@@ -91,3 +92,4 @@ object Correlations {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala
index cb1abbd18fd4d02aa4608742aac0e4e760c1082c..69988cc1b933442f738da62ad4d1b51eb9060729 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scopt.OptionParser
@@ -106,3 +107,4 @@ object CosineSimilarity {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
index 520893b26d595ba911cb5189921bcd32dcf0a210..dc13f82488af79f61a28c60bb23a2961ab4b65aa 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import java.io.File
@@ -119,3 +120,4 @@ object DatasetExample {
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
index 3381941673db84693dc01e7148ff3d32bc9443c5..57ffe3dd2524f9b531e8693d18f71bf3a32c794d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scala.language.reflectiveCalls
@@ -368,3 +369,4 @@ object DecisionTreeRunner {
   }
   // scalastyle:on structural.type
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala
index f8c71ccabc43b97eada8217a2ec09de5553345d9..1fce4ba7efd60ca23d50c1d6312f4857116ef0a8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -65,3 +66,4 @@ object DenseGaussianMixture {
     println()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
index 14cc5cbb679c58cda0e14ddf1a64fb9d82634c20..380d85d60e7b44539eb36d6dee57811e447926cd 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.log4j.{Level, Logger}
@@ -107,3 +108,4 @@ object DenseKMeans {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
index 13f24a1e5961049b9e8fbaee7ded051f4fe3fdbf..14b930550d554301d7d8cd7193b22a87f3520ccb 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scopt.OptionParser
@@ -80,3 +81,4 @@ object FPGrowthExample {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala
index 7416fb5a408484b62ace0f900a7062d036876cc8..e16a6bf03357430dede3fadf4f03e032ad38ea3f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scopt.OptionParser
@@ -145,3 +146,4 @@ object GradientBoostedTreesRunner {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
index 31d629f8531617c3e26f221492ca3a66e38398f1..75b0f69cf91aa3c8062fd0c008b0080c66c6fd13 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import java.text.BreakIterator
@@ -302,3 +303,4 @@ private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Se
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
index 6a456ba7ec07bb2731bbc10a06bb0a491d8ea403..8878061a0970b5b5a9a4e8c41564b31b90b62a5a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.log4j.{Level, Logger}
@@ -134,3 +135,4 @@ object LinearRegression {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
index 99588b0984ab27c880acb0c385d36f3d23e5006f..e43a6f2864c73be3ad0a9d665725cf5ac27224bf 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scala.collection.mutable
@@ -189,3 +190,4 @@ object MovieLensALS {
     math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).mean())
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala
index 6e4e2d07f284b895b754a3149ff99ab54ce2b34e..5f839c75dd5811138ae18f50176413d8b76b5115 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import scopt.OptionParser
@@ -97,3 +98,4 @@ object MultivariateSummarizer {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala
index 6d8b806569dfdf8af58d7bca91c930728e6e9834..0723223954610cde9aac03e722c7c41e5803fcdb 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.log4j.{Level, Logger}
@@ -154,4 +155,4 @@ object PowerIterationClusteringExample {
     coeff * math.exp(expCoeff * ssquares)
   }
 }
-
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala
index 924b586e3af99834efe9504880c115065c5084c5..bee85ba0f9969ad35346dfac6ee51bf739789136 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.mllib.random.RandomRDDs
@@ -58,3 +59,4 @@ object RandomRDDGeneration {
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
index 663c12734af6828d86dd0c108f326891b43f603d..6963f43e082c4f27053a55100f84518ec2216e5e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.mllib.util.MLUtils
@@ -125,3 +126,4 @@ object SampledRDDs {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
index f1ff4e6911f5e42e46da1cda45feddeb8a89a122..f81fc292a3bd1b64d2418ff651b9f672e06eddd9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.log4j.{Level, Logger}
@@ -100,3 +101,4 @@ object SparseNaiveBayes {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala
index 8bb12d2ee9ed228b5f4304186ef3e042402466ad..af03724a8ac621ae7fec9fcc124e713767a9c28a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.SparkConf
@@ -75,3 +76,4 @@ object StreamingKMeansExample {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
index 1a95048bbfe2df96d7c2998cf0bc56a398086a8e..b4a5dca031abdbf28851d9253475c8c380ebb66e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.mllib.linalg.Vectors
@@ -69,3 +70,4 @@ object StreamingLinearRegression {
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala
index e1998099c2d781b32a4f048f1bf42322956a2343..b42f4cb5f9338168e6c7794fe8a22efd92d49e3d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.mllib.linalg.Vectors
@@ -71,3 +72,4 @@ object StreamingLogisticRegression {
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala
index 3cd9cb743e3099ceb0c5ed4b00a484d4404f98c0..464fbd385ab5d9785a3c7dfb0e0da28d7d41ac57 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -58,3 +59,4 @@ object TallSkinnyPCA {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala
index 4d6690318615a5b8b776ccc00354fd3dc7d2eca5..65b4bc46f026615631cdf48e5c85f7cf2449f4fc 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.mllib
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -58,3 +59,4 @@ object TallSkinnySVD {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index b11e32047dc34bb34a04f42b0011804589d7b9df..2cc56f04e5c1f4af9b1120f8c6b4014cf48938cc 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.sql
 
 import org.apache.spark.{SparkConf, SparkContext}
@@ -73,3 +74,4 @@ object RDDRelation {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
index b7ba60ec28155d974372e724e5e4a1bc49ae9605..bf40bd1ef13dfa59426dc1da20e523535fab22a1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.sql.hive
 
 import com.google.common.io.{ByteStreams, Files}
@@ -77,3 +78,4 @@ object HiveFromSpark {
     sc.stop()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
index 016de4c63d1d22f66079748c025d9464f084f820..e9c99071987694bd9abc7e5a1ffa6423eebc713e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import scala.collection.mutable.LinkedList
@@ -170,3 +171,4 @@ object ActorWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
index 30269a7ccae9756238cbcf48a7545b196230f6a8..28e9bf520e5682e9fa02d900212b5a01fede5437 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import java.io.{InputStreamReader, BufferedReader, InputStream}
@@ -100,3 +101,4 @@ class CustomReceiver(host: String, port: Int)
    }
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
index fbe394de4a179eaaa8e0eb2c4584db3ce946724f..bd78526f8c2996dea1c6aed28e1d90e9775f84d4 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import kafka.serializer.StringDecoder
@@ -70,3 +71,4 @@ object DirectKafkaWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
index 20e7df7c45b1bf6f22e070e2eb7333f9396815c1..91e52e4eff5a783806e71e075da8c76128d8daba 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -66,3 +67,4 @@ object FlumeEventCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala
index 1cc8c8d5c23b69ed930b155a7cb5cb175a979aeb..2bdbc37e2a289e6f3bbac8df5298966d302df59a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -65,3 +66,4 @@ object FlumePollingEventCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
index 4b4667fec44e6a2b165eb90ed0b52d76d70d99a8..1f282d437dc38b156f9a90c571b6031a64cbb8bf 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -53,3 +54,4 @@ object HdfsWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
index 60416ee343544400345bca818995a15fc9910f8b..b40d17e9c2fa3234854d801cc6e2e5ba56fa25f5 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import java.util.HashMap
@@ -101,3 +102,4 @@ object KafkaWordCountProducer {
   }
 
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
index 813c8554f5193417ca505ed97599a972a839e12c..d772ae309f40d4058e98de86fdaad3e23eb5afa9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.eclipse.paho.client.mqttv3._
@@ -96,8 +97,10 @@ object MQTTWordCount {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       System.err.println(
         "Usage: MQTTWordCount <MqttbrokerUrl> <topic>")
+      // scalastyle:on println
       System.exit(1)
     }
 
@@ -113,3 +116,4 @@ object MQTTWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
index 2cd8073dada1442dba45d1dbdace3b1b2b737d53..9a57fe286d1ae212ea1fe117411d97407f81f5d1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -57,3 +58,4 @@ object NetworkWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
index a9aaa445bccb6558f72df1edd821838f5817234f..5322929d177b4ae8fd5b889aaede73a98630f245 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -58,3 +59,4 @@ object RawNetworkGrep {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
index 751b30ea1578236daa7e623e2c92324e007796c1..9916882e4f94a2b47885d2a23fb3ed267bbbd98c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import java.io.File
@@ -108,3 +109,4 @@ object RecoverableNetworkWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
index 5a6b9216a3fbcc2bc2fa04a02261fbc7c0143252..ed617754cbf1cd5997b90450ba10a3953318b3e8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -99,3 +100,4 @@ object SQLContextSingleton {
     instance
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
index 345d0bc4413518f14dfc18742b394a7975b1efe9..02ba1c2eed0f7a46876c0366ea8dd5d8c85a4841 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.SparkConf
@@ -78,3 +79,4 @@ object StatefulNetworkWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
index c10de84a80ffe9b5f14924925eb0c28b94533cf5..825c671a929b13825568bbf2691453eebe4a52e3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import com.twitter.algebird._
@@ -113,3 +114,4 @@ object TwitterAlgebirdCMS {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
index 62db5e663b8af9c05974d6a2abea5421db33d4a9..49826ede704187c74f7d6016a6aff84d9d439917 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import com.twitter.algebird.HyperLogLogMonoid
@@ -90,3 +91,4 @@ object TwitterAlgebirdHLL {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
index f253d75b279f75afc4ae79b7726dcf9b91c9e867..49cee1b43c2dc6792c74da3e32e1dc5231dd2f3c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import org.apache.spark.streaming.{Seconds, StreamingContext}
@@ -82,3 +83,4 @@ object TwitterPopularTags {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
index e99d1baa72b9f02b66fd52a9274439ad91d445e0..6ac9a72c37941450d634dd63039f6cb9b7902201 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import akka.actor.ActorSystem
@@ -97,3 +98,4 @@ object ZeroMQWordCount {
     ssc.awaitTermination()
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
index 889f052c70263c2e4ab344b2a13f0ee7ac1b03bc..bea7a47cb285542d107f56a2edf45063391ae372 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming.clickstream
 
 import java.net.ServerSocket
@@ -108,3 +109,4 @@ object PageViewGenerator {
     }
   }
 }
+// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
index fbacaee98690f59691945a6a462aea0efc08a056..ec7d39da8b2e90e0848f7e794a0ca4352b9fdbba 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming.clickstream
 
 import org.apache.spark.SparkContext._
@@ -107,3 +108,4 @@ object PageViewStream {
     ssc.start()
   }
 }
+// scalastyle:on println
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
index 8e1715f6dbb95a4c0886ac374b5e46623822a519..5b3c79444aa6848b4d6fe20578153007445ae6ce 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
@@ -111,7 +111,7 @@ class DirectKafkaStreamSuite
       rdd
     }.foreachRDD { rdd =>
       for (o <- offsetRanges) {
-        println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}")
+        logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}")
       }
       val collected = rdd.mapPartitionsWithIndex { (i, iter) =>
       // For each partition, get size of the range in the partition,
diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala
index be8b62d3cc6baf580f1ca54a370a2971c2a04943..de749626ec09cc5de87fb276a71c22194f4f4736 100644
--- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala
+++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+// scalastyle:off println
 package org.apache.spark.examples.streaming
 
 import java.nio.ByteBuffer
@@ -272,3 +273,4 @@ private[streaming] object StreamingExamples extends Logging {
     }
   }
 }
+// scalastyle:on println
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
index be6b9047d932d66422229c490d177e8009ef67a8..5c07b415cd7962414a458638c9c5fc2ec3dea37f 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
@@ -66,7 +66,6 @@ private[graphx] object BytecodeUtils {
       val finder = new MethodInvocationFinder(c.getName, m)
       getClassReader(c).accept(finder, 0)
       for (classMethod <- finder.methodsInvoked) {
-        // println(classMethod)
         if (classMethod._1 == targetClass && classMethod._2 == targetMethod) {
           return true
         } else if (!seen.contains(classMethod)) {
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
index 9591c4e9b8f4e4246f74bd755089159a14bdea53..989e226305265c1a8e8aa2c36e03cfdd7257d369 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
@@ -33,7 +33,7 @@ import org.apache.spark.graphx.Edge
 import org.apache.spark.graphx.impl.GraphImpl
 
 /** A collection of graph generating functions. */
-object GraphGenerators {
+object GraphGenerators extends Logging {
 
   val RMATa = 0.45
   val RMATb = 0.15
@@ -142,7 +142,7 @@ object GraphGenerators {
     var edges: Set[Edge[Int]] = Set()
     while (edges.size < numEdges) {
       if (edges.size % 100 == 0) {
-        println(edges.size + " edges")
+        logDebug(edges.size + " edges")
       }
       edges += addEdge(numVertices)
     }
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
index 186d0cc2a977be436b31241553386f2d020efeea..61e44dcab578c38af65630a1b11210c49f1f9175 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.graphx.util
 import org.apache.spark.SparkFunSuite
 
 
+// scalastyle:off println
 class BytecodeUtilsSuite extends SparkFunSuite {
 
   import BytecodeUtilsSuite.TestClass
@@ -102,6 +103,7 @@ class BytecodeUtilsSuite extends SparkFunSuite {
   private val c = {e: TestClass => println(e.baz)}
 }
 
+// scalastyle:on println
 
 object BytecodeUtilsSuite {
   class TestClass(val foo: Int, val bar: Long) {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
index 6eaebaf7dba9f04179e9d57ba7efc43bf2a8ec18..e6bcff48b022c024b7b2c9a8b2550f580164cb86 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
@@ -64,8 +64,10 @@ object KMeansDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 6) {
+      // scalastyle:off println
       println("Usage: KMeansGenerator " +
         "<master> <output_dir> <num_points> <k> <d> <r> [<num_partitions>]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index b4e33c98ba7e5ce52fff56f9276d14da73150207..87eeb5db05d26a44c0a35d37b2c2d8c493fefe81 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
@@ -153,8 +153,10 @@ object LinearDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: LinearDataGenerator " +
         "<master> <output_dir> [num_examples] [num_features] [num_partitions]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
index 9d802678c4a771d132ee53993c0660eff7e47f7b..c09cbe69bb97117202ef65e8e4a34b9addf8560b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
@@ -64,8 +64,10 @@ object LogisticRegressionDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length != 5) {
+      // scalastyle:off println
       println("Usage: LogisticRegressionGenerator " +
         "<master> <output_dir> <num_examples> <num_features> <num_partitions>")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index bd73a866c8a82854d912ac876c4721b8dea916ea..16f430599a515c7e19d83b13cda732b92e58a3a7 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -55,8 +55,10 @@ import org.apache.spark.rdd.RDD
 object MFDataGenerator {
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: MFDataGenerator " +
         "<master> <outputDir> [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index a8e30cc9d730cf1a5745503ca3f68bf1140fb604..ad20b7694a7792fdb67c0e95d439f0730107b2da 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
@@ -37,8 +37,10 @@ object SVMDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: SVMGenerator " +
         "<master> <output_dir> [num_examples] [num_features] [num_partitions]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
index 8c85c96d5c6d8ea83cddebe1adcf3bdf7c463b62..03120c828ca96af3a56d897a39b8f51d90567c7c 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ml.feature
 
 import scala.beans.{BeanInfo, BeanProperty}
 
-import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.{Logging, SparkException, SparkFunSuite}
 import org.apache.spark.ml.attribute._
 import org.apache.spark.ml.param.ParamsSuite
 import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors}
@@ -27,7 +27,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.DataFrame
 
-class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
+class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   import VectorIndexerSuite.FeatureData
 
@@ -113,11 +113,11 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
     model.transform(sparsePoints1) // should work
     intercept[SparkException] {
       model.transform(densePoints2).collect()
-      println("Did not throw error when fit, transform were called on vectors of different lengths")
+      logInfo("Did not throw error when fit, transform were called on vectors of different lengths")
     }
     intercept[SparkException] {
       vectorIndexer.fit(badPoints)
-      println("Did not throw error when fitting vectors of different lengths in same RDD.")
+      logInfo("Did not throw error when fitting vectors of different lengths in same RDD.")
     }
   }
 
@@ -196,7 +196,7 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
         }
       } catch {
         case e: org.scalatest.exceptions.TestFailedException =>
-          println(errMsg)
+          logError(errMsg)
           throw e
       }
     }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
index c4ae0a16f7c0445de6ba62926fa5f2fd7b306ecc..178d95a7b94ec67e04449821294e83d18f76e4a2 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
@@ -21,10 +21,10 @@ import scala.util.Random
 
 import breeze.linalg.{DenseMatrix => BDM, squaredDistance => breezeSquaredDistance}
 
-import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.{Logging, SparkException, SparkFunSuite}
 import org.apache.spark.mllib.util.TestingUtils._
 
-class VectorsSuite extends SparkFunSuite {
+class VectorsSuite extends SparkFunSuite with Logging {
 
   val arr = Array(0.1, 0.0, 0.3, 0.4)
   val n = 4
@@ -142,7 +142,7 @@ class VectorsSuite extends SparkFunSuite {
     malformatted.foreach { s =>
       intercept[SparkException] {
         Vectors.parse(s)
-        println(s"Didn't detect malformatted string $s.")
+        logInfo(s"Didn't detect malformatted string $s.")
       }
     }
   }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
index c292ced75e870b32c092b399400c5020493d375d..c3eeda012571c1b81e8e857942d211dccdb3ce7b 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
@@ -19,13 +19,13 @@ package org.apache.spark.mllib.stat
 
 import breeze.linalg.{DenseMatrix => BDM, Matrix => BM}
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.mllib.linalg.Vectors
 import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation,
   SpearmanCorrelation}
 import org.apache.spark.mllib.util.MLlibTestSparkContext
 
-class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext {
+class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   // test input data
   val xData = Array(1.0, 0.0, -2.0)
@@ -146,7 +146,7 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext {
   def matrixApproxEqual(A: BM[Double], B: BM[Double], threshold: Double = 1e-6): Boolean = {
     for (i <- 0 until A.rows; j <- 0 until A.cols) {
       if (!approxEqual(A(i, j), B(i, j), threshold)) {
-        println("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j))
+        logInfo("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j))
         return false
       }
     }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
index 84dd3b342d4c038b4e455d0885d51f89d258cf94..2521b3342181a5fe4926e80a40d7ba2e94a85580 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.mllib.tree
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.tree.configuration.Algo._
 import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy}
@@ -31,7 +31,7 @@ import org.apache.spark.util.Utils
 /**
  * Test suite for [[GradientBoostedTrees]].
  */
-class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext {
+class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   test("Regression with continuous features: SquaredError") {
     GradientBoostedTreesSuite.testCombinations.foreach {
@@ -50,7 +50,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06)
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
@@ -80,7 +80,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.85, "mae")
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
@@ -111,7 +111,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateClassifier(gbt, GradientBoostedTreesSuite.data, 0.9)
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
index fa4f74d71b7e7db9292945ca18f9a396be41f1ba..16d7c3ab39b03abf139bad24fe157889898ee24a 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
@@ -33,7 +33,7 @@ class NumericParserSuite extends SparkFunSuite {
     malformatted.foreach { s =>
       intercept[SparkException] {
         NumericParser.parse(s)
-        println(s"Didn't detect malformatted string $s.")
+        throw new RuntimeException(s"Didn't detect malformatted string $s.")
       }
     }
   }
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 3408c6d51ed4c54f5d4a68dac098d7508f3ef877..4291b0be2a616608376cbcd8a2f34daea59e7c7a 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -69,6 +69,7 @@ object SparkBuild extends PomBuild {
     import scala.collection.mutable
     var isAlphaYarn = false
     var profiles: mutable.Seq[String] = mutable.Seq("sbt")
+    // scalastyle:off println
     if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) {
       println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.")
       profiles ++= Seq("spark-ganglia-lgpl")
@@ -88,6 +89,7 @@ object SparkBuild extends PomBuild {
       println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.")
       profiles ++= Seq("yarn")
     }
+    // scalastyle:on println
     profiles
   }
 
@@ -96,8 +98,10 @@ object SparkBuild extends PomBuild {
     case None => backwardCompatibility
     case Some(v) =>
       if (backwardCompatibility.nonEmpty)
+        // scalastyle:off println
         println("Note: We ignore environment variables, when use of profile is detected in " +
           "conjunction with environment variable.")
+        // scalastyle:on println
       v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq
     }
 
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
index 6480e2d24e044c0fdfa9ac36d1dd9fcf8f1fea44..24fbbc12c08da84dff8f54bf713773033d2c0821 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
@@ -39,6 +39,8 @@ class SparkCommandLine(args: List[String], override val settings: Settings)
   }
 
   def this(args: List[String]) {
+    // scalastyle:off println
     this(args, str => Console.println("Error: " + str))
+    // scalastyle:on println
   }
 }
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 2b235525250c21b0422151b5b980b6c375956b02..8f7f9074d3f0306bd9352f958fc47e15e22af013 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -1101,7 +1101,9 @@ object SparkILoop extends Logging {
             val s = super.readLine()
             // helping out by printing the line being interpreted.
             if (s != null)
+              // scalastyle:off println
               output.println(s)
+              // scalastyle:on println
             s
           }
         }
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
index 05faef8786d2c4c008a3f8affa2e241016c3fde5..bd3314d94eed68858db78d2cd57fcbcd343855e9 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
@@ -80,11 +80,13 @@ private[repl] trait SparkILoopInit {
     if (!initIsComplete)
       withLock { while (!initIsComplete) initLoopCondition.await() }
     if (initError != null) {
+      // scalastyle:off println
       println("""
         |Failed to initialize the REPL due to an unexpected error.
         |This is a bug, please, report it along with the error diagnostics printed below.
         |%s.""".stripMargin.format(initError)
       )
+      // scalastyle:on println
       false
     } else true
   }
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 35fb6256450226e5b446e68910e8fe67c5e6c22f..8791618bd355e2aad67f5cfdab904fd8d5e935ef 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -1761,7 +1761,9 @@ object SparkIMain {
         if (intp.totalSilence) ()
         else super.printMessage(msg)
       }
+      // scalastyle:off println
       else Console.println(msg)
+      // scalastyle:on println
     }
   }
 }
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 7a5e94da5cbf3008c0b9c713092e4e3f261dab57..3c90287249497e71e6e3985f52007f1c4bafd74a 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -943,7 +943,9 @@ object SparkILoop {
             val s = super.readLine()
             // helping out by printing the line being interpreted.
             if (s != null)
+              // scalastyle:off println
               output.println(s)
+              // scalastyle:on println
             s
           }
         }
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 1cb910f376060f03935fdf839971aafd0ec6cdab..56c009a4e38e76a346da7dc15339633dfa888b20 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -129,7 +129,9 @@ class SparkIMain(@BeanProperty val factory: ScriptEngineFactory, initialSettings
   }
   private def tquoted(s: String) = "\"\"\"" + s + "\"\"\""
   private val logScope = scala.sys.props contains "scala.repl.scope"
+  // scalastyle:off println
   private def scopelog(msg: String) = if (logScope) Console.err.println(msg)
+  // scalastyle:on println
 
   // argument is a thunk to execute after init is done
   def initialize(postInitSignal: => Unit) {
@@ -1297,8 +1299,10 @@ class SparkISettings(intp: SparkIMain) {
   def deprecation_=(x: Boolean) = {
     val old = intp.settings.deprecation.value
     intp.settings.deprecation.value = x
+    // scalastyle:off println
     if (!old && x) println("Enabled -deprecation output.")
     else if (old && !x) println("Disabled -deprecation output.")
+    // scalastyle:on println
   }
   def deprecation: Boolean = intp.settings.deprecation.value
 
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkReplReporter.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkReplReporter.scala
index 0711ed4871bb6760e7af6e02196dc4a72d6bb6d0..272f81eca92c15cfd5c9f381e4d0636b54cb2025 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkReplReporter.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkReplReporter.scala
@@ -42,7 +42,9 @@ class SparkReplReporter(intp: SparkIMain) extends ConsoleReporter(intp.settings,
       }
       else super.printMessage(msg)
     }
+    // scalastyle:off println
     else Console.println("[init] " + msg)
+    // scalastyle:on println
   }
 
   override def displayPrompt() {
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index d6f927b6fa803e737629610c5bca3d75f98718f4..49611703798e8dfb9a642ff3a839e26391447212 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -141,12 +141,8 @@ This file is divided into 3 sections:
     <customMessage>Tests must extend org.apache.spark.SparkFunSuite instead.</customMessage>
   </check>
 
-  <!-- ================================================================================ -->
-  <!--       rules we'd like to enforce, but haven't cleaned up the codebase yet        -->
-  <!-- ================================================================================ -->
-
-  <!-- SPARK-7977 We should turn this on, but we'd need to add whitelist to files that are using it first. -->
-  <check customId="println" level="error" class="org.scalastyle.scalariform.TokenChecker" enabled="false">
+  <!-- As of SPARK-7977 all printlns need to be wrapped in '// scalastyle:off/on println' -->
+  <check customId="println" level="error" class="org.scalastyle.scalariform.TokenChecker" enabled="true">
     <parameters><parameter name="regex">^println$</parameter></parameters>
     <customMessage><![CDATA[Are you sure you want to println? If yes, wrap the code block with
       // scalastyle:off println
@@ -154,6 +150,10 @@ This file is divided into 3 sections:
       // scalastyle:on println]]></customMessage>
   </check>
 
+  <!-- ================================================================================ -->
+  <!--       rules we'd like to enforce, but haven't cleaned up the codebase yet        -->
+  <!-- ================================================================================ -->
+
   <!-- We cannot turn the following two on, because it'd fail a lot of string interpolation use cases. -->
   <!-- Ideally the following two rules should be configurable to rule out string interpolation. -->
   <check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="false"></check>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
index 7f1b12cdd580035e5f2fa6f8df6bbb16fa31d59f..606fecbe06e47062a3a5fa8779e4702f6b773e4b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala
@@ -67,8 +67,10 @@ package object codegen {
       outfile.write(generatedBytes)
       outfile.close()
 
+      // scalastyle:off println
       println(
         s"javap -p -v -classpath ${dumpDirectory.getCanonicalPath} ${generatedClass.getName}".!!)
+      // scalastyle:on println
     }
   }
 }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 2f545bb43216559a1af55c931f53730fbf774491..b89e3382f06a9a9bf0d285e5170f754a08ad9569 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -154,7 +154,9 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
   def schemaString: String = schema.treeString
 
   /** Prints out the schema in the tree format */
+  // scalastyle:off println
   def printSchema(): Unit = println(schemaString)
+  // scalastyle:on println
 
   /**
    * A prefix string used when printing the plan.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index 07054166a5e884b3632eb034701951327b19dc99..71293475ca0f92b09f1a8ec338a73bf30f6c0ab6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -124,7 +124,9 @@ package object util {
     val startTime = System.nanoTime()
     val ret = f
     val endTime = System.nanoTime()
+    // scalastyle:off println
     println(s"${(endTime - startTime).toDouble / 1000000}ms")
+    // scalastyle:on println
     ret
   }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
index e0b8ff91786a7ebb67a92fc65ac1557f0eaab443..b8097403ec3cc70328127c49b0dc43e75b0d9a2c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -250,7 +250,9 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
     builder.toString()
   }
 
+  // scalastyle:off println
   def printTreeString(): Unit = println(treeString)
+  // scalastyle:on println
 
   private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
     fields.foreach(field => field.buildFormattedString(prefix, builder))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
index f201c8ea8a1106a6f13d252f2d3e38296208f078..10250264625b2c209cd500093630faa6566cb828 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
@@ -860,11 +860,13 @@ class Column(protected[sql] val expr: Expression) extends Logging {
    * @since 1.3.0
    */
   def explain(extended: Boolean): Unit = {
+    // scalastyle:off println
     if (extended) {
       println(expr)
     } else {
       println(expr.prettyString)
     }
+    // scalastyle:on println
   }
 
   /**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index d7966651b19484a8b74ae69860fb7d910f790bbf..830fba35bb7bc48be69aa7098478ca9aae2d657b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -308,7 +308,9 @@ class DataFrame private[sql](
    * @group basic
    * @since 1.3.0
    */
+  // scalastyle:off println
   def printSchema(): Unit = println(schema.treeString)
+  // scalastyle:on println
 
   /**
    * Prints the plans (logical and physical) to the console for debugging purposes.
@@ -319,7 +321,9 @@ class DataFrame private[sql](
     ExplainCommand(
       queryExecution.logical,
       extended = extended).queryExecution.executedPlan.executeCollect().map {
+      // scalastyle:off println
       r => println(r.getString(0))
+      // scalastyle:on println
     }
   }
 
@@ -392,7 +396,9 @@ class DataFrame private[sql](
    * @group action
    * @since 1.5.0
    */
+  // scalastyle:off println
   def show(numRows: Int, truncate: Boolean): Unit = println(showString(numRows, truncate))
+  // scalastyle:on println
 
   /**
    * Returns a [[DataFrameNaFunctions]] for working with missing data.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index 2964edac1aba2274e6b68c7f022f384c501e60b3..e6081cb05bc2d534e67a0135aa39e9bfa4bf1502 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -24,7 +24,7 @@ import org.apache.spark.unsafe.types.UTF8String
 
 import scala.collection.mutable.HashSet
 
-import org.apache.spark.{AccumulatorParam, Accumulator}
+import org.apache.spark.{AccumulatorParam, Accumulator, Logging}
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.trees.TreeNodeRef
@@ -57,7 +57,7 @@ package object debug {
    * Augments [[DataFrame]]s with debug methods.
    */
   @DeveloperApi
-  implicit class DebugQuery(query: DataFrame) {
+  implicit class DebugQuery(query: DataFrame) extends Logging {
     def debug(): Unit = {
       val plan = query.queryExecution.executedPlan
       val visited = new collection.mutable.HashSet[TreeNodeRef]()
@@ -66,7 +66,7 @@ package object debug {
           visited += new TreeNodeRef(s)
           DebugNode(s)
       }
-      println(s"Results returned: ${debugPlan.execute().count()}")
+      logDebug(s"Results returned: ${debugPlan.execute().count()}")
       debugPlan.foreach {
         case d: DebugNode => d.dumpStats()
         case _ =>
@@ -82,11 +82,11 @@ package object debug {
           TypeCheck(s)
       }
       try {
-        println(s"Results returned: ${debugPlan.execute().count()}")
+        logDebug(s"Results returned: ${debugPlan.execute().count()}")
       } catch {
         case e: Exception =>
           def unwrap(e: Throwable): Throwable = if (e.getCause == null) e else unwrap(e.getCause)
-          println(s"Deepest Error: ${unwrap(e)}")
+          logDebug(s"Deepest Error: ${unwrap(e)}")
       }
     }
   }
@@ -119,11 +119,11 @@ package object debug {
     val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics())
 
     def dumpStats(): Unit = {
-      println(s"== ${child.simpleString} ==")
-      println(s"Tuples output: ${tupleCount.value}")
+      logDebug(s"== ${child.simpleString} ==")
+      logDebug(s"Tuples output: ${tupleCount.value}")
       child.output.zip(columnStats).foreach { case(attr, metric) =>
         val actualDataTypes = metric.elementTypes.value.mkString("{", ",", "}")
-        println(s" ${attr.name} ${attr.dataType}: $actualDataTypes")
+        logDebug(s" ${attr.name} ${attr.dataType}: $actualDataTypes")
       }
     }
 
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index 039cfa40d26b32228f9ffa04af04f4f6a8bfdc0b..f66a17b20915f376347da582c0b82a7390a1b266 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -40,7 +40,7 @@ import org.apache.spark.Logging
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.util.Utils
 
-private[hive] object SparkSQLCLIDriver {
+private[hive] object SparkSQLCLIDriver extends Logging {
   private var prompt = "spark-sql"
   private var continuedPrompt = "".padTo(prompt.length, ' ')
   private var transport: TSocket = _
@@ -164,7 +164,7 @@ private[hive] object SparkSQLCLIDriver {
       }
     } catch {
       case e: FileNotFoundException =>
-        System.err.println(s"Could not open input file for reading. (${e.getMessage})")
+        logError(s"Could not open input file for reading. (${e.getMessage})")
         System.exit(3)
     }
 
@@ -180,14 +180,14 @@ private[hive] object SparkSQLCLIDriver {
         val historyFile = historyDirectory + File.separator + ".hivehistory"
         reader.setHistory(new History(new File(historyFile)))
       } else {
-        System.err.println("WARNING: Directory for Hive history file: " + historyDirectory +
+        logWarning("WARNING: Directory for Hive history file: " + historyDirectory +
                            " does not exist.   History will not be available during this session.")
       }
     } catch {
       case e: Exception =>
-        System.err.println("WARNING: Encountered an error while trying to initialize Hive's " +
+        logWarning("WARNING: Encountered an error while trying to initialize Hive's " +
                            "history file.  History will not be available during this session.")
-        System.err.println(e.getMessage)
+        logWarning(e.getMessage)
     }
 
     val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport")
@@ -270,6 +270,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
       val proc: CommandProcessor = CommandProcessorFactory.get(Array(tokens(0)), hconf)
 
       if (proc != null) {
+        // scalastyle:off println
         if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] ||
           proc.isInstanceOf[AddResourceProcessor]) {
           val driver = new SparkSQLDriver
@@ -336,6 +337,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
           }
           ret = proc.run(cmd_1).getResponseCode
         }
+        // scalastyle:on println
       }
       ret
     }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index bbc39b892b79e8c5b9fa7a7391d0f1fcead80eaa..4684d48aff88950127edc001bc4a5e2ea7c094f8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.parse.VariableSubstitution
 import org.apache.hadoop.hive.ql.session.SessionState
 import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable}
 
+import org.apache.spark.Logging
 import org.apache.spark.SparkContext
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql._
@@ -65,12 +66,12 @@ private[hive] class HiveQLDialect extends ParserDialect {
  *
  * @since 1.0.0
  */
-class HiveContext(sc: SparkContext) extends SQLContext(sc) {
+class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging {
   self =>
 
   import HiveContext._
 
-  println("create HiveContext")
+  logDebug("create HiveContext")
 
   /**
    * When true, enables an experimental feature where metastore tables that use the parquet SerDe
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 2de7a99c122fd25c3b25738adb071200ae659da2..7fc517b646b201f3c92777b7c2c02c92a2205309 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.parse._
 import org.apache.hadoop.hive.ql.plan.PlanUtils
 import org.apache.hadoop.hive.ql.session.SessionState
 
+import org.apache.spark.Logging
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions._
@@ -73,7 +74,7 @@ private[hive] case class CreateTableAsSelect(
 }
 
 /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */
-private[hive] object HiveQl {
+private[hive] object HiveQl extends Logging {
   protected val nativeCommands = Seq(
     "TOK_ALTERDATABASE_OWNER",
     "TOK_ALTERDATABASE_PROPERTIES",
@@ -186,7 +187,7 @@ private[hive] object HiveQl {
             .map(ast => Option(ast).map(_.transform(rule)).orNull))
       } catch {
         case e: Exception =>
-          println(dumpTree(n))
+          logError(dumpTree(n).toString)
           throw e
       }
     }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index cbd2bf6b5eede20f39c8f1833b6477353db66762..9d83ca6c113dca783dc6c9075ce37481e3a5c4fa 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -360,7 +360,9 @@ private[hive] class ClientWrapper(
 
         case _ =>
           if (state.out != null) {
+            // scalastyle:off println
             state.out.println(tokens(0) + " " + cmd_1)
+            // scalastyle:on println
           }
           Seq(proc.run(cmd_1).getResponseCode.toString)
       }
diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala
index 0e428ba1d74569180e6159eac7335a6124319156..2590040f2ec1ca13c7cbf58de61b17c09a615e81 100644
--- a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala
+++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.hive.HiveContext
  */
 object Main {
   def main(args: Array[String]) {
+    // scalastyle:off println
     println("Running regression test for SPARK-8489.")
     val sc = new SparkContext("local", "testing")
     val hc = new HiveContext(sc)
@@ -38,6 +39,7 @@ object Main {
     val df = hc.createDataFrame(Seq(MyCoolClass("1", "2", "3")))
     df.collect()
     println("Regression test for SPARK-8489 success!")
+    // scalastyle:on println
     sc.stop()
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index e9bb32667936cad311c7d759e6482eca4dc15d84..983c013bcf86aeb28dcae0cac551253b33169d78 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -17,13 +17,13 @@
 
 package org.apache.spark.sql.hive
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.sql.hive.test.TestHive
 
 import org.apache.spark.sql.test.ExamplePointUDT
 import org.apache.spark.sql.types.StructType
 
-class HiveMetastoreCatalogSuite extends SparkFunSuite {
+class HiveMetastoreCatalogSuite extends SparkFunSuite with Logging {
 
   test("struct field should accept underscore in sub-column name") {
     val metastr = "struct<a: int, b_1: string, c: string>"
@@ -41,7 +41,7 @@ class HiveMetastoreCatalogSuite extends SparkFunSuite {
   test("duplicated metastore relations") {
     import TestHive.implicits._
     val df = TestHive.sql("SELECT * FROM src")
-    println(df.queryExecution)
+    logInfo(df.queryExecution.toString)
     df.as('a).join(df.as('b), $"a.key" === $"b.key")
   }
 }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index a38ed23b5cf9a4fd6503d9e4d482b34e011beb84..917900e5f46dc23aee5b7d5fe73a44c8354aa15a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -90,8 +90,10 @@ class HiveSparkSubmitSuite
       "SPARK_TESTING" -> "1",
       "SPARK_HOME" -> sparkHome
     ).run(ProcessLogger(
+      // scalastyle:off println
       (line: String) => { println(s"out> $line") },
       (line: String) => { println(s"err> $line") }
+      // scalastyle:on println
     ))
 
     try {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
index aa5dbe2db690339009846e127a0bb4d90fdafeea..508695919e9a70e0b6c6280b9fe6cda90ba129c6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala
@@ -86,8 +86,6 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter {
     val message = intercept[QueryExecutionException] {
       sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)")
     }.getMessage
-
-    println("message!!!!" + message)
   }
 
   test("Double create does not fail when allowExisting = true") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index cc294bc3e8bc3cfb6974ba4050cc758c33a0000e..d910af22c3dd1b8b3e9eb144336a69871d24908b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.mapred.InvalidInputException
 
+import org.apache.spark.Logging
 import org.apache.spark.sql._
 import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable}
 import org.apache.spark.sql.hive.test.TestHive
@@ -40,7 +41,8 @@ import org.apache.spark.util.Utils
 /**
  * Tests for persisting tables created though the data sources API into the metastore.
  */
-class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeAndAfterAll {
+class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeAndAfterAll
+  with Logging {
   override val sqlContext = TestHive
 
   var jsonFilePath: String = _
@@ -415,7 +417,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
            |)
          """.stripMargin)
 
-      sql("DROP TABLE jsonTable").collect().foreach(println)
+      sql("DROP TABLE jsonTable").collect().foreach(i => logInfo(i.toString))
     }
   }
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index eaaa88e17002bfa821093d1415ebc7b8aa6d2f1a..1bde5922b52789078fea326a26c5002f3d49f7c5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -315,7 +315,6 @@ class PairUDF extends GenericUDF {
   )
 
   override def evaluate(args: Array[DeferredObject]): AnyRef = {
-    println("Type = %s".format(args(0).getClass.getName))
     Integer.valueOf(args(0).get.asInstanceOf[TestPair].entry._2)
   }
 
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
index 192aa6a139bcb9866b46c5cbb2c2e594bbac8e75..1da0b0a54df07174912658336b0b22eb9b9ecd22 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
@@ -720,12 +720,14 @@ abstract class DStream[T: ClassTag] (
     def foreachFunc: (RDD[T], Time) => Unit = {
       (rdd: RDD[T], time: Time) => {
         val firstNum = rdd.take(num + 1)
+        // scalastyle:off println
         println("-------------------------------------------")
         println("Time: " + time)
         println("-------------------------------------------")
         firstNum.take(num).foreach(println)
         if (firstNum.length > num) println("...")
         println()
+        // scalastyle:on println
       }
     }
     new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index ca2f319f174a2f9807db92cdaf4202fbc4d5fa76..6addb9675203816d5c07896ac4c43acc9ad52465 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -35,7 +35,9 @@ private[streaming]
 object RawTextSender extends Logging {
   def main(args: Array[String]) {
     if (args.length != 4) {
+      // scalastyle:off println
       System.err.println("Usage: RawTextSender <port> <file> <blockSize> <bytesPerSec>")
+      // scalastyle:on println
       System.exit(1)
     }
     // Parse the arguments using a pattern match
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
index c8eef833eb4312b43e3a561c0c83f2bba018f113..dd32ad5ad811d6d2f9c1d66435d98d6e3a359868 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
@@ -106,7 +106,7 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name:
 }
 
 private[streaming]
-object RecurringTimer {
+object RecurringTimer extends Logging {
 
   def main(args: Array[String]) {
     var lastRecurTime = 0L
@@ -114,7 +114,7 @@ object RecurringTimer {
 
     def onRecur(time: Long) {
       val currentTime = System.currentTimeMillis()
-      println("" + currentTime + ": " + (currentTime - lastRecurTime))
+      logInfo("" + currentTime + ": " + (currentTime - lastRecurTime))
       lastRecurTime = currentTime
     }
     val timer = new  RecurringTimer(new SystemClock(), period, onRecur, "Test")
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
index e0f14fd954280604c01152b2c85d1e466af27ab0..6e9d4431090a2bc0d337fa11dd8d18df959b1109 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala
@@ -43,6 +43,7 @@ object MasterFailureTest extends Logging {
   @volatile var setupCalled = false
 
   def main(args: Array[String]) {
+    // scalastyle:off println
     if (args.size < 2) {
       println(
         "Usage: MasterFailureTest <local/HDFS directory> <# batches> " +
@@ -60,6 +61,7 @@ object MasterFailureTest extends Logging {
     testUpdateStateByKey(directory, numBatches, batchDuration)
 
     println("\n\nSUCCESS\n\n")
+    // scalastyle:on println
   }
 
   def testMap(directory: String, numBatches: Int, batchDuration: Duration) {
@@ -291,10 +293,12 @@ object MasterFailureTest extends Logging {
     }
 
     // Log the output
+    // scalastyle:off println
     println("Expected output, size = " + expectedOutput.size)
     println(expectedOutput.mkString("[", ",", "]"))
     println("Output, size = " + output.size)
     println(output.mkString("[", ",", "]"))
+    // scalastyle:on println
 
     // Match the output with the expected output
     output.foreach(o =>
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala
index 7865b06c2e3c2d7ef36131847bfd2d22aa0c05c6..a2dbae149f311794e01e008bd86ad883e07cc767 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala
@@ -76,7 +76,6 @@ class JobGeneratorSuite extends TestSuiteBase {
         if (time.milliseconds == longBatchTime) {
           while (waitLatch.getCount() > 0) {
             waitLatch.await()
-            println("Await over")
           }
         }
       })
diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
index 595ded6ae67fab4c4a845945d2fe9d0e67288f53..9483d2b692ab55231750057faea79d82a9e3ce00 100644
--- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala
@@ -92,7 +92,9 @@ object GenerateMIMAIgnore {
         ignoredMembers ++= getAnnotatedOrPackagePrivateMembers(classSymbol)
 
       } catch {
+        // scalastyle:off println
         case _: Throwable => println("Error instrumenting class:" + className)
+        // scalastyle:on println
       }
     }
     (ignoredClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet, ignoredMembers.toSet)
@@ -108,7 +110,9 @@ object GenerateMIMAIgnore {
         .filter(_.contains("$$")).map(classSymbol.fullName + "." + _)
     } catch {
       case t: Throwable =>
+        // scalastyle:off println
         println("[WARN] Unable to detect inner functions for class:" + classSymbol.fullName)
+        // scalastyle:on println
         Seq.empty[String]
     }
   }
@@ -128,12 +132,14 @@ object GenerateMIMAIgnore {
       getOrElse(Iterator.empty).mkString("\n")
     File(".generated-mima-class-excludes")
       .writeAll(previousContents + privateClasses.mkString("\n"))
+    // scalastyle:off println
     println("Created : .generated-mima-class-excludes in current directory.")
     val previousMembersContents = Try(File(".generated-mima-member-excludes").lines)
       .getOrElse(Iterator.empty).mkString("\n")
     File(".generated-mima-member-excludes").writeAll(previousMembersContents +
       privateMembers.mkString("\n"))
     println("Created : .generated-mima-member-excludes in current directory.")
+    // scalastyle:on println
   }
 
 
@@ -174,7 +180,9 @@ object GenerateMIMAIgnore {
       try {
         classes += Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader)
       } catch {
+        // scalastyle:off println
         case _: Throwable => println("Unable to load:" + entry)
+        // scalastyle:on println
       }
     }
     classes
diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
index 583823c90c5c6219b60ef4c2a01eac89f66b6b11..856ea177a9a1078ba418c8b93a68da63d608e6db 100644
--- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
@@ -323,11 +323,14 @@ object JavaAPICompletenessChecker {
     val missingMethods = javaEquivalents -- javaMethods
 
     for (method <- missingMethods) {
+      // scalastyle:off println
       println(method)
+      // scalastyle:on println
     }
   }
 
   def main(args: Array[String]) {
+    // scalastyle:off println
     println("Missing RDD methods")
     printMissingMethods(classOf[RDD[_]], classOf[JavaRDD[_]])
     println()
@@ -359,5 +362,6 @@ object JavaAPICompletenessChecker {
     println("Missing PairDStream methods")
     printMissingMethods(classOf[PairDStreamFunctions[_, _]], classOf[JavaPairDStream[_, _]])
     println()
+    // scalastyle:on println
   }
 }
diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala
index baa97616eaff3da737cc7f16f5b069f1156cbb0c..0dc2861253f17a19efa04e06e8721b34fb7769f0 100644
--- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala
@@ -85,7 +85,9 @@ object StoragePerfTester {
             latch.countDown()
           } catch {
             case e: Exception =>
+              // scalastyle:off println
               println("Exception in child thread: " + e + " " + e.getMessage)
+              // scalastyle:on println
               System.exit(1)
           }
         }
@@ -97,9 +99,11 @@ object StoragePerfTester {
     val bytesPerSecond = totalBytes.get() / time
     val bytesPerFile = (totalBytes.get() / (numOutputSplits * numMaps.toDouble)).toLong
 
+    // scalastyle:off println
     System.err.println("files_total\t\t%s".format(numMaps * numOutputSplits))
     System.err.println("bytes_per_file\t\t%s".format(Utils.bytesToString(bytesPerFile)))
     System.err.println("agg_throughput\t\t%s/s".format(Utils.bytesToString(bytesPerSecond.toLong)))
+    // scalastyle:on println
 
     executor.shutdown()
     sc.stop()
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
index 68e9f6b4db7f4bbd27722983d9c8d3e1c085cc7e..37f793763367e895c3be184f0cdc1202853c5965 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -85,7 +85,9 @@ class ApplicationMasterArguments(val args: Array[String]) {
     }
 
     if (primaryPyFile != null && primaryRFile != null) {
+      // scalastyle:off println
       System.err.println("Cannot have primary-py-file and primary-r-file at the same time")
+      // scalastyle:on println
       System.exit(-1)
     }
 
@@ -93,6 +95,7 @@ class ApplicationMasterArguments(val args: Array[String]) {
   }
 
   def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+    // scalastyle:off println
     if (unknownParam != null) {
       System.err.println("Unknown/unsupported param " + unknownParam)
     }
@@ -111,6 +114,7 @@ class ApplicationMasterArguments(val args: Array[String]) {
       |  --executor-cores NUM   Number of cores for the executors (Default: 1)
       |  --executor-memory MEM  Memory per executor (e.g. 1000M, 2G) (Default: 1G)
       """.stripMargin)
+    // scalastyle:on println
     System.exit(exitCode)
   }
 }
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 4d52ae774ea003e1509eb6647121231d6b280391..f0af6f875f5235c7ccfedd6fc6be21e5afb54108 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -938,7 +938,7 @@ private[spark] class Client(
 object Client extends Logging {
   def main(argStrings: Array[String]) {
     if (!sys.props.contains("SPARK_SUBMIT")) {
-      println("WARNING: This client is deprecated and will be removed in a " +
+      logWarning("WARNING: This client is deprecated and will be removed in a " +
         "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"")
     }
 
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 19d1bbff9993ff52a0da8e77f1eb6d4c04b4fd53..20d63d40cf605ee4feee0f7311b011feb2900541 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -123,6 +123,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
       throw new SparkException("Executor cores must not be less than " +
         "spark.task.cpus.")
     }
+    // scalastyle:off println
     if (isClusterMode) {
       for (key <- Seq(amMemKey, amMemOverheadKey, amCoresKey)) {
         if (sparkConf.contains(key)) {
@@ -144,11 +145,13 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
         .map(_.toInt)
         .foreach { cores => amCores = cores }
     }
+    // scalastyle:on println
   }
 
   private def parseArgs(inputArgs: List[String]): Unit = {
     var args = inputArgs
 
+    // scalastyle:off println
     while (!args.isEmpty) {
       args match {
         case ("--jar") :: value :: tail =>
@@ -253,6 +256,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf)
           throw new IllegalArgumentException(getUsageMessage(args))
       }
     }
+    // scalastyle:on println
 
     if (primaryPyFile != null && primaryRFile != null) {
       throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" +
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
index 335e966519c7ca7ced4052159ce936f3faa86b64..547863d9a07399744cf5f8868ce63b2991c79014 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala
@@ -328,12 +328,14 @@ private object YarnClusterDriver extends Logging with Matchers {
 
   def main(args: Array[String]): Unit = {
     if (args.length != 1) {
+      // scalastyle:off println
       System.err.println(
         s"""
         |Invalid command line: ${args.mkString(" ")}
         |
         |Usage: YarnClusterDriver [result file]
         """.stripMargin)
+      // scalastyle:on println
       System.exit(1)
     }
 
@@ -386,12 +388,14 @@ private object YarnClasspathTest {
 
   def main(args: Array[String]): Unit = {
     if (args.length != 2) {
+      // scalastyle:off println
       System.err.println(
         s"""
         |Invalid command line: ${args.mkString(" ")}
         |
         |Usage: YarnClasspathTest [driver result file] [executor result file]
         """.stripMargin)
+      // scalastyle:on println
       System.exit(1)
     }