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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 0 additions & 2 deletions core/src/main/scala/org/apache/spark/SecurityManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,9 @@ import javax.net.ssl._
import com.google.common.hash.HashCodes
import com.google.common.io.Files
import org.apache.hadoop.io.Text
import org.apache.hadoop.security.Credentials

import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.network.sasl.SecretKeyHolder
import org.apache.spark.util.Utils

Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark

import java.io._
import java.lang.reflect.Constructor
import java.net.{URI}
import java.net.URI
import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID}
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.deploy

import javax.annotation.concurrent.ThreadSafe

import com.codahale.metrics.{Gauge, MetricRegistry}
import com.codahale.metrics.MetricRegistry

import org.apache.spark.metrics.source.Source
import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.deploy

import java.io.{File, IOException, PrintStream}
import java.io.{File, IOException}
import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException}
import java.net.URL
import java.security.PrivilegedExceptionAction
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,6 @@ import javax.servlet.http.HttpServletRequest

import scala.xml.{Node, Unparsed}

import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}

import org.apache.spark.internal.Logging
import org.apache.spark.ui.{UIUtils, WebUIPage}
import org.apache.spark.util.Utils
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,6 @@

package org.apache.spark.internal.config

import java.util.{Map => JMap}

import scala.util.matching.Regex

import org.apache.spark.SparkConf

/**
* An entry contains all meta information for a configuration.
*
Expand All @@ -34,7 +28,6 @@ import org.apache.spark.SparkConf
* value declared as a string.
*
* @param key the key for the configuration
* @param defaultValue the default value for the configuration
* @param valueConverter how to convert a string to the value. It should throw an exception if the
* string does not have the required format.
* @param stringConverter how to convert a value to a string that the user can use it as a valid
Expand Down Expand Up @@ -76,7 +69,7 @@ private class ConfigEntryWithDefault[T] (
stringConverter: T => String,
doc: String,
isPublic: Boolean)
extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) {
extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) {

override def defaultValue: Option[T] = Some(_defaultValue)

Expand All @@ -95,7 +88,7 @@ private class ConfigEntryWithDefaultString[T] (
stringConverter: T => String,
doc: String,
isPublic: Boolean)
extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) {
extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic) {

override def defaultValue: Option[T] = Some(valueConverter(_defaultValue))

Expand All @@ -118,8 +111,8 @@ private[spark] class OptionalConfigEntry[T](
val rawStringConverter: T => String,
doc: String,
isPublic: Boolean)
extends ConfigEntry[Option[T]](key, s => Some(rawValueConverter(s)),
v => v.map(rawStringConverter).orNull, doc, isPublic) {
extends ConfigEntry[Option[T]](key, s => Some(rawValueConverter(s)),
v => v.map(rawStringConverter).orNull, doc, isPublic) {

override def defaultValueString: String = "<undefined>"

Expand All @@ -137,7 +130,7 @@ private class FallbackConfigEntry[T] (
doc: String,
isPublic: Boolean,
private[config] val fallback: ConfigEntry[T])
extends ConfigEntry[T](key, fallback.valueConverter, fallback.stringConverter, doc, isPublic) {
extends ConfigEntry[T](key, fallback.valueConverter, fallback.stringConverter, doc, isPublic) {

override def defaultValueString: String = s"<value of ${fallback.key}>"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.spark.internal.config

import java.util.{Map => JMap}
import java.util.regex.Pattern

import scala.collection.mutable.HashMap
import scala.util.matching.Regex
Expand Down
5 changes: 2 additions & 3 deletions core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,10 @@ package org.apache.spark.rpc

import java.util.concurrent.TimeoutException

import scala.concurrent.{Await, Future}
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.control.NonFatal

import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.SparkConf
import org.apache.spark.util.{ThreadUtils, Utils}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.rpc.netty
import java.io.File
import java.util.concurrent.ConcurrentHashMap

import org.apache.spark.SparkContext
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Was this intentional? I would have guessed this wouldn't compile without this import before if it were needed.

import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.server.StreamManager
import org.apache.spark.rpc.RpcEnvFileServer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import java.util.Properties

import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.rdd.RDD

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import scala.language.existentials

import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.shuffle.ShuffleWriter
Expand Down
1 change: 0 additions & 1 deletion core/src/main/scala/org/apache/spark/scheduler/Stage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.scheduler

import scala.collection.mutable.HashSet

import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
Expand Down
1 change: 0 additions & 1 deletion core/src/main/scala/org/apache/spark/scheduler/Task.scala
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@ import org.apache.spark.util._
* @param stageId id of the stage this task belongs to
* @param stageAttemptId attempt id of the stage this task belongs to
* @param partitionId index of the number in the RDD
* @param metrics a `TaskMetrics` that is created at driver side and sent to executor side.
* @param localProperties copy of thread-local properties set by the user on the driver side.
* @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side
* and sent to executor side.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import javax.annotation.concurrent.NotThreadSafe

import scala.reflect.ClassTag

import org.apache.spark.SparkEnv
import org.apache.spark.annotation.{DeveloperApi, Private}
import org.apache.spark.util.NextIterator

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.nio.ByteBuffer
import scala.reflect.ClassTag

import org.apache.spark.SparkConf
import org.apache.spark.internal.config._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.security.CryptoStreamUtils
import org.apache.spark.storage._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.spark.status.api.v1

import javax.ws.rs.{GET, PathParam, Produces}
import javax.ws.rs.{GET, Produces}
import javax.ws.rs.core.MediaType

import org.apache.spark.ui.SparkUI
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils
* ExternalBlockStore, whether to keep the data in memory in a serialized format, and whether
* to replicate the RDD partitions on multiple nodes.
*
* The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants
* The [[org.apache.spark.storage.StorageLevel]] singleton object contains some static constants
* for commonly useful storage levels. To create your own storage level object, use the
* factory method of the singleton object (`StorageLevel(...)`).
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.util.random

import java.util.Random

import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag

import org.apache.commons.math3.distribution.PoissonDistribution
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark

import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer

import org.apache.spark.executor.TaskMetrics
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import org.mockito.Mockito.mock
import org.scalatest._

import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
import org.apache.spark.internal.config._
import org.apache.spark.network.BlockDataManager

class NettyBlockTransferServiceSuite
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,6 @@ import org.scalatest.Matchers

import org.apache.spark.SparkFunSuite

/**
*
*/

class DistributionSuite extends SparkFunSuite with Matchers {
test("summary") {
val d = new Distribution((1 to 100).toArray.map{_.toDouble})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.apache.spark.examples.ml
// $example on$
import org.apache.spark.ml.feature.Binarizer
// $example off$
import org.apache.spark.sql.{SparkSession}
import org.apache.spark.sql.SparkSession

object BinarizerExample {
def main(args: Array[String]): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,6 @@
*/
package org.apache.spark.examples.sql

// $example on:schema_inferring$
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.Encoder
// $example off:schema_inferring$
import org.apache.spark.sql.Row
// $example on:init_session$
import org.apache.spark.sql.SparkSession
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
// scalastyle:off println
package org.apache.spark.examples.sql.streaming

import org.apache.spark.sql.functions._
import org.apache.spark.sql.SparkSession

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets
import scala.collection.JavaConverters._
import scala.util.control.NonFatal

import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetOutOfRangeException}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer}
import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
import org.apache.kafka.common.TopicPartition

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord }
import org.apache.kafka.common.TopicPartition

import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext}
import org.apache.spark.{Partition, SparkContext, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.rdd.RDD
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import org.apache.kafka.common.TopicPartition
import org.apache.spark.SparkContext
import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext }
import org.apache.spark.api.java.function.{ Function0 => JFunction0 }
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.StreamingContext
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.nio.ByteBuffer

import scala.util.Random

import com.amazonaws.auth.{BasicAWSCredentials, DefaultAWSCredentialsProviderChain}
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain
import com.amazonaws.regions.RegionUtils
import com.amazonaws.services.kinesis.AmazonKinesisClient
import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason
import org.apache.spark.internal.Logging
import org.apache.spark.streaming.Duration
import org.apache.spark.streaming.util.RecurringTimer
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils}
import org.apache.spark.util.{Clock, SystemClock}

/**
* This is a helper class for managing Kinesis checkpointing.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.streaming.kinesis

import java.util.concurrent.{ExecutorService, TimeoutException}
import java.util.concurrent.TimeoutException

import scala.concurrent.{Await, ExecutionContext, Future}
import scala.concurrent.duration._
Expand All @@ -30,7 +30,6 @@ import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester}
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.Eventually._
import org.scalatest.mock.MockitoSugar

import org.apache.spark.streaming.{Duration, TestSuiteBase}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,8 @@

import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;

import static org.apache.spark.launcher.SparkSubmitOptionParser.*;

public class SparkSubmitOptionParserSuite extends BaseSuite {

private SparkSubmitOptionParser parser;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Dataset, Row}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.{DataType, DoubleType, StructType}
import org.apache.spark.sql.types.{DataType, StructType}

/**
* (private[spark]) Params for classification.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTMod
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Dataset, Row}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.DoubleType

/**
* Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}

import org.apache.spark.TaskContext
import org.apache.spark.ml.feature.LabeledPoint
import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT}
import org.apache.spark.ml.linalg.{Vectors, VectorUDT}
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.spark.repl

import scala.tools.nsc.{Settings, CompilerCommand}
import scala.Predef._
import scala.tools.nsc.{CompilerCommand, Settings}

import org.apache.spark.annotation.DeveloperApi

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,8 +10,6 @@ package org.apache.spark.repl
import scala.tools.nsc._
import scala.tools.nsc.interpreter._

import scala.reflect.internal.util.Position
import scala.util.control.Exception.ignoring
import scala.tools.nsc.util.stackTraceString

import org.apache.spark.SPARK_VERSION
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 scala.collection.mutable.ArrayBuffer

import org.apache.spark.util.{IntParam, MemoryParam}

class ApplicationMasterArguments(val args: Array[String]) {
var userJar: String = null
var userClass: String = null
Expand Down
Loading