Skip to content

Commit

Permalink
[SPARK-16779][TRIVIAL] Avoid using postfix operators where they do no…
Browse files Browse the repository at this point in the history
…t add much and remove whitelisting

## What changes were proposed in this pull request?

Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.

## How was this patch tested?

Existing tests.

Author: Holden Karau <[email protected]>

Closes apache#14407 from holdenk/SPARK-16779.
  • Loading branch information
holdenk authored and Marcelo Vanzin committed Aug 8, 2016
1 parent 8650239 commit 9216901
Show file tree
Hide file tree
Showing 21 changed files with 6 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.{Arrays, Comparator, Date}

import scala.collection.JavaConverters._
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.control.NonFatal

import com.google.common.primitives.Longs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet
import scala.language.postfixOps
import scala.util.Random

import org.apache.spark._
Expand Down
2 changes: 0 additions & 2 deletions core/src/main/scala/org/apache/spark/util/RpcUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.util

import scala.language.postfixOps

import org.apache.spark.SparkConf
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import java.util.concurrent.{ExecutorService, TimeUnit}
import scala.collection.Map
import scala.collection.mutable
import scala.concurrent.duration._
import scala.language.postfixOps

import org.mockito.Matchers
import org.mockito.Matchers._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import javax.servlet.http.{HttpServletRequest, HttpServletResponse}

import scala.collection.mutable
import scala.collection.mutable.ListBuffer
import scala.language.postfixOps

import com.codahale.metrics.Counter
import com.google.common.cache.LoadingCache
Expand Down
4 changes: 2 additions & 2 deletions core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import java.io.File

import scala.collection.Map
import scala.io.Codec
import scala.language.postfixOps
import scala.sys.process._
import scala.util.Try

Expand Down Expand Up @@ -215,7 +214,8 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
}

def testCommandAvailable(command: String): Boolean = {
Try(Process(command) !!).isSuccess
val attempt = Try(Process(command).run().exitValue())
attempt.isSuccess && attempt.get == 0
}

def testExportInputFile(varName: String) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ package org.apache.spark.storage
import java.nio.ByteBuffer

import scala.language.implicitConversions
import scala.language.postfixOps
import scala.language.reflectiveCalls
import scala.reflect.ClassTag

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.concurrent.TimeoutException

import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.language.postfixOps
import scala.util.control.NonFatal

import kafka.admin.AdminUtils
Expand Down Expand Up @@ -279,4 +278,3 @@ private[kafka010] class KafkaTestUtils extends Logging {
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.concurrent.TimeoutException

import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.language.postfixOps
import scala.util.control.NonFatal

import kafka.admin.AdminUtils
Expand Down Expand Up @@ -274,4 +273,3 @@ private[kafka] class KafkaTestUtils extends Logging {
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.graphx.lib

import scala.language.postfixOps
import scala.reflect.ClassTag

import org.apache.spark.graphx._
Expand Down Expand Up @@ -109,7 +108,7 @@ object PageRank extends Logging {
require(resetProb >= 0 && resetProb <= 1, s"Random reset probability must belong" +
s" to [0, 1], but got ${resetProb}")

val personalized = srcId isDefined
val personalized = srcId.isDefined
val src: VertexId = srcId.getOrElse(-1L)

// Initialize the PageRank graph with each edge attribute having
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.mllib.util

import java.{util => ju}

import scala.language.postfixOps
import scala.util.Random

import org.apache.spark.SparkContext
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import java.util
import scala.concurrent.duration._
import scala.io.Source
import scala.language.implicitConversions
import scala.language.postfixOps

import com.google.common.io.Files
import org.mockito.Matchers.anyString
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql.types

import scala.language.postfixOps

import org.scalatest.PrivateMethodTester

import org.apache.spark.SparkFunSuite
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import java.io.File
import java.nio.charset.StandardCharsets
import java.util.UUID

import scala.language.postfixOps
import scala.util.Random

import org.scalatest.Matchers._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql

import scala.language.postfixOps

import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.expressions.scalalang.typed
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql

import scala.language.postfixOps

import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSQLContext

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql

import scala.language.postfixOps

import org.apache.spark.sql.test.SharedSQLContext

case class IntClass(value: Int)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@ package org.apache.spark.sql
import java.io.{Externalizable, ObjectInput, ObjectOutput}
import java.sql.{Date, Timestamp}

import scala.language.postfixOps

import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder}
import org.apache.spark.sql.catalyst.util.sideBySide
import org.apache.spark.sql.execution.streaming.MemoryStream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1790,6 +1790,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}

def testCommandAvailable(command: String): Boolean = {
Try(Process(command) !!).isSuccess
val attempt = Try(Process(command).run().exitValue())
attempt.isSuccess && attempt.get == 0
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicInteger

import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.language.postfixOps

import com.google.common.io.Files
import org.apache.hadoop.fs.Path
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@ package org.apache.spark.deploy.yarn
import java.security.PrivilegedExceptionAction
import java.util.concurrent.{Executors, TimeUnit}

import scala.language.postfixOps

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.security.UserGroupInformation
Expand Down Expand Up @@ -128,7 +126,7 @@ private[yarn] class AMDelegationTokenRenewer(
try {
val remoteFs = FileSystem.get(freshHadoopConf)
val credentialsPath = new Path(credentialsFile)
val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles days).toMillis
val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles.days).toMillis
hadoopUtil.listFilesSorted(
remoteFs, credentialsPath.getParent,
credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION)
Expand Down

0 comments on commit 9216901

Please sign in to comment.