From a719184c92acd513a4646fb8baaeda7660a74656 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 17 Mar 2016 00:04:07 +0800 Subject: [PATCH 1/3] Move org.apache.spark.Logging into org.apache.spark.internal.Logging --- core/src/main/scala/org/apache/spark/Accumulator.scala | 1 + core/src/main/scala/org/apache/spark/ContextCleaner.scala | 1 + .../scala/org/apache/spark/ExecutorAllocationManager.scala | 1 + .../src/main/scala/org/apache/spark/HeartbeatReceiver.scala | 1 + core/src/main/scala/org/apache/spark/HttpServer.scala | 1 + core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 1 + core/src/main/scala/org/apache/spark/SSLOptions.scala | 2 ++ core/src/main/scala/org/apache/spark/SecurityManager.scala | 1 + core/src/main/scala/org/apache/spark/SparkConf.scala | 4 ++-- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + core/src/main/scala/org/apache/spark/SparkEnv.scala | 1 + .../src/main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 + core/src/main/scala/org/apache/spark/TaskContextImpl.scala | 1 + core/src/main/scala/org/apache/spark/TaskEndReason.scala | 1 + .../org/apache/spark/api/python/PythonGatewayServer.scala | 2 +- .../org/apache/spark/api/python/PythonHadoopUtil.scala | 3 ++- .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 1 + .../org/apache/spark/api/python/PythonWorkerFactory.scala | 1 + .../main/scala/org/apache/spark/api/python/SerDeUtil.scala | 3 ++- core/src/main/scala/org/apache/spark/api/r/RBackend.scala | 3 ++- .../main/scala/org/apache/spark/api/r/RBackendHandler.scala | 2 +- core/src/main/scala/org/apache/spark/api/r/RRDD.scala | 1 + .../main/scala/org/apache/spark/broadcast/Broadcast.scala | 2 +- .../scala/org/apache/spark/broadcast/BroadcastManager.scala | 3 ++- .../scala/org/apache/spark/broadcast/TorrentBroadcast.scala | 1 + core/src/main/scala/org/apache/spark/deploy/Client.scala | 5 +++-- .../org/apache/spark/deploy/ExternalShuffleService.scala | 3 ++- .../scala/org/apache/spark/deploy/FaultToleranceTest.scala | 3 ++- .../scala/org/apache/spark/deploy/LocalSparkCluster.scala | 3 ++- .../main/scala/org/apache/spark/deploy/RPackageUtils.scala | 2 +- .../scala/org/apache/spark/deploy/SparkCuratorUtil.scala | 3 ++- .../scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 3 ++- .../scala/org/apache/spark/deploy/client/AppClient.scala | 5 +++-- .../org/apache/spark/deploy/history/ApplicationCache.scala | 2 +- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 3 ++- .../org/apache/spark/deploy/history/HistoryServer.scala | 3 ++- .../spark/deploy/history/HistoryServerArguments.scala | 3 ++- .../spark/deploy/master/FileSystemPersistenceEngine.scala | 2 +- .../main/scala/org/apache/spark/deploy/master/Master.scala | 3 ++- .../apache/spark/deploy/master/RecoveryModeFactory.scala | 3 ++- .../spark/deploy/master/ZooKeeperLeaderElectionAgent.scala | 3 ++- .../spark/deploy/master/ZooKeeperPersistenceEngine.scala | 3 ++- .../org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../apache/spark/deploy/mesos/MesosClusterDispatcher.scala | 3 ++- .../spark/deploy/mesos/MesosExternalShuffleService.scala | 3 ++- .../org/apache/spark/deploy/rest/RestSubmissionClient.scala | 3 ++- .../org/apache/spark/deploy/rest/RestSubmissionServer.scala | 3 ++- .../scala/org/apache/spark/deploy/worker/CommandUtils.scala | 2 +- .../scala/org/apache/spark/deploy/worker/DriverRunner.scala | 3 ++- .../org/apache/spark/deploy/worker/ExecutorRunner.scala | 3 ++- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 3 ++- .../org/apache/spark/deploy/worker/WorkerWatcher.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ui/LogPage.scala | 2 +- .../org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- .../spark/executor/CoarseGrainedExecutorBackend.scala | 1 + .../src/main/scala/org/apache/spark/executor/Executor.scala | 1 + .../org/apache/spark/executor/MesosExecutorBackend.scala | 3 ++- .../main/scala/org/apache/spark/executor/TaskMetrics.scala | 1 + .../apache/spark/input/FixedLengthBinaryInputFormat.scala | 2 +- .../scala/org/apache/spark/{ => internal}/Logging.scala | 2 +- .../org/apache/spark/mapred/SparkHadoopMapRedUtil.scala | 3 ++- .../scala/org/apache/spark/memory/ExecutionMemoryPool.scala | 2 +- .../main/scala/org/apache/spark/memory/MemoryManager.scala | 3 ++- .../scala/org/apache/spark/memory/StorageMemoryPool.scala | 2 +- .../main/scala/org/apache/spark/metrics/MetricsConfig.scala | 3 ++- .../main/scala/org/apache/spark/metrics/MetricsSystem.scala | 3 ++- .../org/apache/spark/network/BlockTransferService.scala | 4 ++-- .../apache/spark/network/netty/NettyBlockRpcServer.scala | 2 +- .../main/scala/org/apache/spark/rdd/AsyncRDDActions.scala | 3 ++- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 3 ++- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala | 3 ++- core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../scala/org/apache/spark/rdd/OrderedRDDFunctions.scala | 6 +++--- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 1 + core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + .../main/scala/org/apache/spark/rdd/RDDOperationScope.scala | 3 ++- .../scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala | 1 + .../org/apache/spark/rdd/ReliableRDDCheckpointData.scala | 1 + .../org/apache/spark/rdd/SequenceFileRDDFunctions.scala | 2 +- .../main/scala/org/apache/spark/rpc/RpcEndpointRef.scala | 3 ++- .../main/scala/org/apache/spark/rpc/netty/Dispatcher.scala | 3 ++- core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala | 3 ++- .../org/apache/spark/rpc/netty/NettyRpcCallContext.scala | 2 +- .../main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala | 3 ++- core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala | 3 ++- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 ++- .../scala/org/apache/spark/scheduler/InputFormatInfo.scala | 2 +- .../main/scala/org/apache/spark/scheduler/JobWaiter.scala | 2 +- .../apache/spark/scheduler/OutputCommitCoordinator.scala | 1 + core/src/main/scala/org/apache/spark/scheduler/Pool.scala | 2 +- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 2 +- .../org/apache/spark/scheduler/SchedulableBuilder.scala | 3 ++- .../scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 1 + .../scala/org/apache/spark/scheduler/SparkListener.scala | 3 ++- core/src/main/scala/org/apache/spark/scheduler/Stage.scala | 1 + .../scala/org/apache/spark/scheduler/TaskResultGetter.scala | 1 + .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 3 ++- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 ++- .../cluster/mesos/MesosClusterPersistenceEngine.scala | 3 ++- .../scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala | 3 ++- .../spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala | 3 ++- .../spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala | 2 +- .../org/apache/spark/scheduler/local/LocalBackend.scala | 3 ++- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 1 + .../org/apache/spark/serializer/SerializationDebugger.scala | 2 +- .../org/apache/spark/shuffle/BlockStoreShuffleReader.scala | 1 + .../org/apache/spark/shuffle/FileShuffleBlockResolver.scala | 3 ++- .../apache/spark/shuffle/IndexShuffleBlockResolver.scala | 3 ++- .../org/apache/spark/shuffle/hash/HashShuffleManager.scala | 1 + .../org/apache/spark/shuffle/hash/HashShuffleWriter.scala | 1 + .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 1 + .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 1 + .../spark/status/api/v1/EventLogDownloadResource.scala | 3 ++- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 3 ++- .../main/scala/org/apache/spark/storage/BlockManager.scala | 1 + .../scala/org/apache/spark/storage/BlockManagerMaster.scala | 3 ++- .../apache/spark/storage/BlockManagerMasterEndpoint.scala | 3 ++- .../apache/spark/storage/BlockManagerSlaveEndpoint.scala | 3 ++- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 3 ++- .../org/apache/spark/storage/DiskBlockObjectWriter.scala | 2 +- .../src/main/scala/org/apache/spark/storage/DiskStore.scala | 3 ++- .../apache/spark/storage/ShuffleBlockFetcherIterator.scala | 3 ++- .../scala/org/apache/spark/storage/memory/MemoryStore.scala | 3 ++- .../main/scala/org/apache/spark/ui/ConsoleProgressBar.scala | 1 + core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 3 ++- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 3 ++- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- core/src/main/scala/org/apache/spark/ui/WebUI.scala | 3 ++- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 1 + .../scala/org/apache/spark/ui/scope/RDDOperationGraph.scala | 2 +- .../main/scala/org/apache/spark/util/ClosureCleaner.scala | 3 ++- core/src/main/scala/org/apache/spark/util/EventLoop.scala | 2 +- core/src/main/scala/org/apache/spark/util/ListenerBus.scala | 2 +- .../scala/org/apache/spark/util/ShutdownHookManager.scala | 2 +- .../main/scala/org/apache/spark/util/SizeEstimator.scala | 2 +- .../apache/spark/util/SparkUncaughtExceptionHandler.scala | 2 +- .../scala/org/apache/spark/util/TimeStampedHashMap.scala | 2 +- core/src/main/scala/org/apache/spark/util/Utils.scala | 1 + .../spark/util/collection/ExternalAppendOnlyMap.scala | 3 ++- .../org/apache/spark/util/collection/ExternalSorter.scala | 1 + .../scala/org/apache/spark/util/collection/Spillable.scala | 3 ++- .../scala/org/apache/spark/util/logging/FileAppender.scala | 3 ++- .../scala/org/apache/spark/util/logging/RollingPolicy.scala | 2 +- .../apache/spark/util/random/StratifiedSamplingUtils.scala | 2 +- .../test/scala/org/apache/spark/ContextCleanerSuite.scala | 1 + .../apache/spark/SparkContextSchedulerCreationSuite.scala | 1 + core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/ThreadingSuite.scala | 2 ++ .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 1 + .../org/apache/spark/deploy/client/AppClientSuite.scala | 1 + .../apache/spark/deploy/history/ApplicationCacheSuite.scala | 3 ++- .../spark/deploy/history/FsHistoryProviderSuite.scala | 3 ++- .../apache/spark/input/WholeTextFileRecordReaderSuite.scala | 3 ++- core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala | 3 ++- .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 1 + .../org/apache/spark/scheduler/TaskSchedulerImplSuite.scala | 1 + .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 1 + .../scala/org/apache/spark/util/FileAppenderSuite.scala | 3 ++- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 3 ++- .../org/apache/spark/util/collection/SorterSuite.scala | 3 ++- .../scala/org/apache/spark/examples/graphx/Analytics.scala | 1 + .../apache/spark/examples/streaming/CustomReceiver.scala | 3 ++- .../apache/spark/examples/streaming/StreamingExamples.scala | 2 +- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 3 ++- .../apache/spark/streaming/kafka/KafkaInputDStream.scala | 2 +- .../scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 3 ++- .../org/apache/spark/streaming/kafka/KafkaTestUtils.scala | 3 ++- .../spark/streaming/kafka/ReliableKafkaReceiver.scala | 3 ++- .../spark/streaming/kafka/DirectKafkaStreamSuite.scala | 3 ++- .../spark/examples/streaming/KinesisWordCountASL.scala | 3 ++- .../spark/streaming/kinesis/KinesisBackedBlockRDD.scala | 1 + .../spark/streaming/kinesis/KinesisCheckpointer.scala | 2 +- .../apache/spark/streaming/kinesis/KinesisReceiver.scala | 2 +- .../spark/streaming/kinesis/KinesisRecordProcessor.scala | 2 +- .../apache/spark/streaming/kinesis/KinesisTestUtils.scala | 2 +- .../main/scala/org/apache/spark/graphx/GraphLoader.scala | 3 ++- graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala | 2 +- .../apache/spark/graphx/impl/VertexPartitionBaseOps.scala | 2 +- .../main/scala/org/apache/spark/graphx/lib/PageRank.scala | 2 +- .../org/apache/spark/graphx/util/GraphGenerators.scala | 1 + mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala | 3 ++- mllib/src/main/scala/org/apache/spark/ml/Transformer.scala | 2 +- .../org/apache/spark/ml/classification/GBTClassifier.scala | 2 +- .../apache/spark/ml/classification/LogisticRegression.scala | 3 ++- .../src/main/scala/org/apache/spark/ml/clustering/LDA.scala | 2 +- .../org/apache/spark/ml/feature/QuantileDiscretizer.scala | 2 +- .../spark/ml/optim/IterativelyReweightedLeastSquares.scala | 2 +- .../org/apache/spark/ml/optim/WeightedLeastSquares.scala | 2 +- .../main/scala/org/apache/spark/ml/recommendation/ALS.scala | 3 ++- .../apache/spark/ml/regression/AFTSurvivalRegression.scala | 3 ++- .../scala/org/apache/spark/ml/regression/GBTRegressor.scala | 2 +- .../spark/ml/regression/GeneralizedLinearRegression.scala | 3 ++- .../org/apache/spark/ml/regression/IsotonicRegression.scala | 2 +- .../org/apache/spark/ml/regression/LinearRegression.scala | 3 ++- .../apache/spark/ml/tree/impl/GradientBoostedTrees.scala | 2 +- .../scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala | 2 +- .../scala/org/apache/spark/ml/tree/impl/RandomForest.scala | 2 +- .../scala/org/apache/spark/ml/tuning/CrossValidator.scala | 3 ++- .../org/apache/spark/ml/tuning/TrainValidationSplit.scala | 2 +- .../src/main/scala/org/apache/spark/ml/util/ReadWrite.scala | 3 ++- .../org/apache/spark/mllib/classification/NaiveBayes.scala | 3 ++- .../org/apache/spark/mllib/clustering/BisectingKMeans.scala | 2 +- .../spark/mllib/clustering/BisectingKMeansModel.scala | 2 +- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 2 +- .../main/scala/org/apache/spark/mllib/clustering/LDA.scala | 2 +- .../org/apache/spark/mllib/clustering/LocalKMeans.scala | 2 +- .../spark/mllib/clustering/PowerIterationClustering.scala | 3 ++- .../org/apache/spark/mllib/clustering/StreamingKMeans.scala | 2 +- .../mllib/evaluation/BinaryClassificationMetrics.scala | 2 +- .../org/apache/spark/mllib/evaluation/RankingMetrics.scala | 2 +- .../apache/spark/mllib/evaluation/RegressionMetrics.scala | 4 ++-- .../org/apache/spark/mllib/feature/StandardScaler.scala | 2 +- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 3 +-- .../scala/org/apache/spark/mllib/fpm/AssociationRules.scala | 2 +- .../main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala | 3 ++- .../scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala | 2 +- .../main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala | 2 +- .../org/apache/spark/mllib/impl/PeriodicCheckpointer.scala | 3 ++- .../src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala | 2 +- .../apache/spark/mllib/linalg/distributed/BlockMatrix.scala | 3 ++- .../apache/spark/mllib/linalg/distributed/RowMatrix.scala | 2 +- .../apache/spark/mllib/optimization/GradientDescent.scala | 2 +- .../scala/org/apache/spark/mllib/optimization/LBFGS.scala | 2 +- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 2 +- .../mllib/recommendation/MatrixFactorizationModel.scala | 3 ++- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 3 ++- .../spark/mllib/regression/StreamingLinearAlgorithm.scala | 2 +- .../spark/mllib/stat/correlation/PearsonCorrelation.scala | 2 +- .../spark/mllib/stat/correlation/SpearmanCorrelation.scala | 2 +- .../scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala | 3 ++- .../spark/mllib/stat/test/KolmogorovSmirnovTest.scala | 2 +- .../org/apache/spark/mllib/stat/test/StreamingTest.scala | 2 +- .../apache/spark/mllib/stat/test/StreamingTestMethod.scala | 2 +- .../scala/org/apache/spark/mllib/tree/DecisionTree.scala | 2 +- .../org/apache/spark/mllib/tree/GradientBoostedTrees.scala | 2 +- .../scala/org/apache/spark/mllib/tree/RandomForest.scala | 2 +- .../apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala | 2 +- .../apache/spark/mllib/tree/model/DecisionTreeModel.scala | 3 ++- .../main/scala/org/apache/spark/mllib/tree/model/Node.scala | 2 +- .../apache/spark/mllib/tree/model/treeEnsembleModels.scala | 3 ++- .../scala/org/apache/spark/mllib/util/DataValidators.scala | 2 +- .../org/apache/spark/ml/feature/VectorIndexerSuite.scala | 3 ++- .../scala/org/apache/spark/ml/recommendation/ALSSuite.scala | 4 ++-- .../scala/org/apache/spark/mllib/linalg/VectorsSuite.scala | 3 ++- .../org/apache/spark/mllib/stat/CorrelationSuite.scala | 3 ++- .../apache/spark/mllib/tree/GradientBoostedTreesSuite.scala | 3 ++- .../src/main/scala/org/apache/spark/repl/Main.scala | 4 +--- .../main/scala/org/apache/spark/repl/SparkExprTyper.scala | 2 +- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 6 ++---- .../src/main/scala/org/apache/spark/repl/SparkIMain.scala | 3 ++- .../scala/org/apache/spark/repl/SparkJLineCompletion.scala | 6 +++--- .../src/main/scala/org/apache/spark/repl/Main.scala | 1 + .../scala/org/apache/spark/repl/ExecutorClassLoader.scala | 3 ++- .../org/apache/spark/repl/ExecutorClassLoaderSuite.scala | 1 + .../spark/sql/catalyst/expressions/BoundAttribute.scala | 2 +- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- .../sql/catalyst/expressions/codegen/GenerateOrdering.scala | 2 +- .../org/apache/spark/sql/catalyst/parser/ParseDriver.scala | 2 +- .../apache/spark/sql/catalyst/planning/QueryPlanner.scala | 2 +- .../org/apache/spark/sql/catalyst/planning/patterns.scala | 2 +- .../spark/sql/catalyst/plans/logical/LogicalPlan.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/rules/Rule.scala | 2 +- .../org/apache/spark/sql/catalyst/rules/RuleExecutor.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/trees/package.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 2 +- .../main/scala/org/apache/spark/sql/DataFrameReader.scala | 3 ++- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 ++- .../main/scala/org/apache/spark/sql/UDFRegistration.scala | 2 +- .../scala/org/apache/spark/sql/execution/CacheManager.scala | 2 +- .../scala/org/apache/spark/sql/execution/SparkPlan.scala | 3 ++- .../spark/sql/execution/aggregate/AggregationIterator.scala | 2 +- .../execution/aggregate/TungstenAggregationIterator.scala | 3 ++- .../sql/execution/aggregate/TypedAggregateExpression.scala | 2 +- .../org/apache/spark/sql/execution/aggregate/udaf.scala | 2 +- .../sql/execution/columnar/GenerateColumnAccessor.scala | 2 +- .../columnar/compression/CompressibleColumnBuilder.scala | 2 +- .../org/apache/spark/sql/execution/command/commands.scala | 2 +- .../scala/org/apache/spark/sql/execution/command/ddl.scala | 2 +- .../apache/spark/sql/execution/datasources/DataSource.scala | 2 +- .../sql/execution/datasources/DataSourceStrategy.scala | 3 ++- .../sql/execution/datasources/FileSourceStrategy.scala | 2 +- .../spark/sql/execution/datasources/SqlNewHadoopRDD.scala | 1 + .../spark/sql/execution/datasources/WriterContainer.scala | 1 + .../spark/sql/execution/datasources/csv/CSVOptions.scala | 2 +- .../spark/sql/execution/datasources/csv/CSVParser.scala | 2 +- .../spark/sql/execution/datasources/csv/CSVRelation.scala | 2 +- .../sql/execution/datasources/jdbc/DriverRegistry.scala | 2 +- .../spark/sql/execution/datasources/jdbc/JDBCRDD.scala | 3 ++- .../spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- .../spark/sql/execution/datasources/json/JSONRelation.scala | 2 +- .../execution/datasources/parquet/CatalystReadSupport.scala | 2 +- .../datasources/parquet/CatalystRowConverter.scala | 2 +- .../datasources/parquet/CatalystWriteSupport.scala | 2 +- .../sql/execution/datasources/parquet/ParquetRelation.scala | 3 ++- .../org/apache/spark/sql/execution/debug/package.scala | 3 ++- .../spark/sql/execution/exchange/ExchangeCoordinator.scala | 3 ++- .../org/apache/spark/sql/execution/python/PythonUDF.scala | 2 +- .../org/apache/spark/sql/execution/stat/FrequentItems.scala | 2 +- .../org/apache/spark/sql/execution/stat/StatFunctions.scala | 2 +- .../spark/sql/execution/streaming/FileStreamSource.scala | 2 +- .../spark/sql/execution/streaming/StreamExecution.scala | 2 +- .../org/apache/spark/sql/execution/streaming/memory.scala | 3 ++- .../apache/spark/sql/execution/ui/AllExecutionsPage.scala | 2 +- .../org/apache/spark/sql/execution/ui/ExecutionPage.scala | 2 +- .../org/apache/spark/sql/execution/ui/SQLListener.scala | 3 ++- .../scala/org/apache/spark/sql/execution/ui/SQLTab.scala | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../scala/org/apache/spark/sql/sources/interfaces.scala | 3 ++- .../org/apache/spark/sql/util/QueryExecutionListener.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameStatSuite.scala | 2 +- .../spark/sql/execution/columnar/ColumnTypeSuite.scala | 3 ++- .../spark/sql/hive/thriftserver/HiveThriftServer2.scala | 3 ++- .../hive/thriftserver/SparkExecuteStatementOperation.scala | 2 +- .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala | 2 +- .../apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala | 3 ++- .../hive/thriftserver/server/SparkSQLOperationManager.scala | 2 +- .../spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 2 +- .../sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 2 +- .../spark/sql/hive/thriftserver/ui/ThriftServerTab.scala | 3 ++- .../main/scala/org/apache/spark/sql/hive/HiveCatalog.scala | 2 +- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 3 ++- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveShim.scala | 2 +- .../main/scala/org/apache/spark/sql/hive/SQLBuilder.scala | 2 +- .../main/scala/org/apache/spark/sql/hive/TableReader.scala | 2 +- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 3 ++- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 2 +- .../apache/spark/sql/hive/client/IsolatedClientLoader.scala | 3 ++- .../spark/sql/hive/execution/ScriptTransformation.scala | 3 ++- .../src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala | 2 +- .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 1 + .../org/apache/spark/sql/hive/orc/OrcFileOperator.scala | 2 +- .../scala/org/apache/spark/sql/hive/orc/OrcFilters.scala | 2 +- .../scala/org/apache/spark/sql/hive/orc/OrcRelation.scala | 2 +- .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 1 + .../org/apache/spark/sql/hive/client/FiltersSuite.scala | 3 ++- .../org/apache/spark/sql/hive/client/VersionsSuite.scala | 3 ++- .../main/scala/org/apache/spark/streaming/Checkpoint.scala | 3 ++- .../scala/org/apache/spark/streaming/DStreamGraph.scala | 2 +- .../scala/org/apache/spark/streaming/StreamingContext.scala | 1 + .../scala/org/apache/spark/streaming/dstream/DStream.scala | 3 ++- .../spark/streaming/dstream/DStreamCheckpointData.scala | 2 +- .../apache/spark/streaming/dstream/RawInputDStream.scala | 2 +- .../apache/spark/streaming/dstream/SocketInputDStream.scala | 2 +- .../apache/spark/streaming/receiver/BlockGenerator.scala | 3 ++- .../org/apache/spark/streaming/receiver/RateLimiter.scala | 3 ++- .../spark/streaming/receiver/ReceivedBlockHandler.scala | 3 ++- .../spark/streaming/receiver/ReceiverSupervisor.scala | 3 ++- .../spark/streaming/receiver/ReceiverSupervisorImpl.scala | 3 ++- .../apache/spark/streaming/scheduler/InputInfoTracker.scala | 2 +- .../org/apache/spark/streaming/scheduler/JobGenerator.scala | 3 ++- .../org/apache/spark/streaming/scheduler/JobScheduler.scala | 2 +- .../spark/streaming/scheduler/ReceivedBlockTracker.scala | 3 ++- .../apache/spark/streaming/scheduler/ReceiverTracker.scala | 1 + .../spark/streaming/scheduler/rate/PIDRateEstimator.scala | 2 +- .../scala/org/apache/spark/streaming/ui/StreamingPage.scala | 2 +- .../scala/org/apache/spark/streaming/ui/StreamingTab.scala | 3 ++- .../apache/spark/streaming/util/BatchedWriteAheadLog.scala | 3 ++- .../spark/streaming/util/FileBasedWriteAheadLog.scala | 3 ++- .../spark/streaming/util/FileBasedWriteAheadLogReader.scala | 2 +- .../spark/streaming/util/RateLimitedOutputStream.scala | 2 +- .../org/apache/spark/streaming/util/RawTextSender.scala | 3 ++- .../org/apache/spark/streaming/util/RecurringTimer.scala | 2 +- .../apache/spark/streaming/util/WriteAheadLogUtils.scala | 3 ++- .../scala/org/apache/spark/streaming/FailureSuite.scala | 1 + .../org/apache/spark/streaming/InputStreamsSuite.scala | 2 +- .../org/apache/spark/streaming/MasterFailureTest.scala | 2 +- .../apache/spark/streaming/ReceivedBlockHandlerSuite.scala | 1 + .../apache/spark/streaming/ReceivedBlockTrackerSuite.scala | 3 ++- .../org/apache/spark/streaming/StreamingContextSuite.scala | 1 + .../org/apache/spark/streaming/StreamingListenerSuite.scala | 2 +- .../scala/org/apache/spark/streaming/TestSuiteBase.scala | 3 ++- .../apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala | 3 ++- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 1 + .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 3 ++- .../spark/deploy/yarn/ClientDistributedCacheManager.scala | 2 +- .../spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala | 3 ++- .../org/apache/spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- .../scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 3 ++- .../scala/org/apache/spark/deploy/yarn/YarnRMClient.scala | 3 ++- .../spark/scheduler/cluster/SchedulerExtensionService.scala | 3 ++- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 3 ++- .../spark/scheduler/cluster/YarnSchedulerBackend.scala | 3 ++- 390 files changed, 557 insertions(+), 340 deletions(-) rename core/src/main/scala/org/apache/spark/{ => internal}/Logging.scala (99%) diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala index 0e4bcc33cba5..985752933a6a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -23,6 +23,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.ref.WeakReference +import org.apache.spark.internal.Logging import org.apache.spark.storage.{BlockId, BlockStatus} diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 17014e4954f9..8fc657c5ebe4 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{ConcurrentLinkedQueue, ScheduledExecutorService, Ti import scala.collection.JavaConverters._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 9b8279f43e75..0926d05414ba 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -24,6 +24,7 @@ import scala.util.control.ControlThrowable import com.codahale.metrics.{Gauge, MetricRegistry} +import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 7f474ed591f9..e8748dd80a37 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable import scala.concurrent.Future +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 3c808420c8b2..9fad1f6786ad 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -28,6 +28,7 @@ import org.eclipse.jetty.servlet.{DefaultServlet, ServletContextHandler, Servlet import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 9cb61597904c..3a5caa3510eb 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.reflect.ClassTag +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.MetadataFetchFailedException diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index d755f07965e6..30db6ccbf4ac 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -23,6 +23,8 @@ import javax.net.ssl.SSLContext import org.eclipse.jetty.util.ssl.SslContextFactory +import org.apache.spark.internal.Logging + /** * SSLOptions class is a common container for SSL configuration options. It offers methods to * generate specific objects to configure SSL for different communication protocols. diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 6132fa349ef2..e8f68224d597 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -28,6 +28,7 @@ import com.google.common.io.Files import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.network.sasl.SecretKeyHolder import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 22e9c5e95631..5da2e98f1f77 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -17,15 +17,15 @@ package org.apache.spark -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable.LinkedHashSet import org.apache.avro.{Schema, SchemaNormalization} +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} -import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5c7ae57b7879..d2cf3bfd60ee 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -50,6 +50,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index b3b3729625ad..878a8c130823 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -28,6 +28,7 @@ import com.google.common.collect.MapMaker import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.internal.Logging import org.apache.spark.memory.{MemoryManager, StaticMemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 58647860623e..17daac173c50 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType +import org.apache.spark.internal.Logging import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD import org.apache.spark.util.SerializableJobConf diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 7e96040bc468..c9354b3e5574 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 509fb2eb0e08..83af226bfd6f 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -23,6 +23,7 @@ import scala.util.Try import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala index 164e95081583..6c4072272572 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala @@ -22,7 +22,7 @@ import java.net.Socket import py4j.GatewayServer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index d2beef2a0dd4..6f6730690f85 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -23,8 +23,9 @@ import scala.util.{Failure, Success, Try} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io._ -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8f306770a184..f423b2ee5626 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -36,6 +36,7 @@ import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 433764be89fb..3df87f62f2f8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import scala.collection.JavaConverters._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.util.{RedirectThread, Utils} private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 55db938f09a9..1c632ebdf925 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -28,8 +28,9 @@ import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ 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 8b3be0da2c8c..9d29a844130f 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 @@ -29,7 +29,8 @@ import io.netty.channel.socket.nio.NioServerSocketChannel import io.netty.handler.codec.LengthFieldBasedFrameDecoder import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging /** * Netty-based backend server that is used to communicate between R and Java. diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 9bddd7248c7e..c416e835a904 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -25,8 +25,8 @@ import scala.language.existentials import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} import io.netty.channel.ChannelHandler.Sharable -import org.apache.spark.Logging import org.apache.spark.api.r.SerDe._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** 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 401f362fee82..588a57e65f55 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 @@ -29,6 +29,7 @@ import scala.util.Try import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 0d68872dcb6e..24d953e77bfa 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -21,8 +21,8 @@ import java.io.Serializable import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index be416c4f74cb..e88988fe03b2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -21,7 +21,8 @@ import java.util.concurrent.atomic.AtomicLong import scala.reflect.ClassTag -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging private[spark] class BroadcastManager( val isDriver: Boolean, diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 8091aa8062a2..2634d8836766 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BroadcastBlockId, StorageLevel} 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 dcef03ef3e3e..640f25f5048c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -22,11 +22,12 @@ import scala.concurrent.ExecutionContext import scala.reflect.ClassTag import scala.util.{Failure, Success} -import org.apache.log4j.{Level, Logger} +import org.apache.log4j.Logger -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index c514a1a86bab..adc0de1e9127 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -21,7 +21,8 @@ import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.sasl.SaslServerBootstrap diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 305994a3f354..abb98f95a1ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -32,8 +32,9 @@ import scala.sys.process._ import org.json4s._ import org.json4s.jackson.JsonMethods -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.master.RecoveryState +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 2dfb813d5fb4..84aa8944fc1c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -19,9 +19,10 @@ package org.apache.spark.deploy import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala index 81718e0c44ec..3d2cabcdfdd5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala @@ -26,8 +26,8 @@ import scala.collection.JavaConverters._ import com.google.common.io.{ByteStreams, Files} -import org.apache.spark.Logging import org.apache.spark.api.r.RUtils +import org.apache.spark.internal.Logging import org.apache.spark.util.{RedirectThread, Utils} private[deploy] object RPackageUtils extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 8d5e716e6aea..8247110940db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -23,7 +23,8 @@ import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry import org.apache.zookeeper.KeeperException -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging private[spark] object SparkCuratorUtil extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 270ca84e24ae..06b7b388ca54 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -35,8 +35,9 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index b9dec62abcca..43b17e5d49bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -23,12 +23,13 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ -import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} +import org.apache.spark.util.{RpcUtils, ThreadUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 000f7e8e1e6e..a370526c46f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -28,7 +28,7 @@ import com.codahale.metrics.{Counter, MetricRegistry, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache, RemovalListener, RemovalNotification} import org.eclipse.jetty.servlet.FilterHolder -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.Source import org.apache.spark.ui.SparkUI import org.apache.spark.util.Clock diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index f88579876046..d5afb33c7118 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -31,8 +31,9 @@ import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 076bdc5c058e..d821474bdb59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,8 +25,9 @@ import scala.util.control.NonFatal import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ 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 fc3790f8d79b..2eddb5ff5447 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 @@ -19,7 +19,8 @@ package org.apache.spark.deploy.history import scala.annotation.tailrec -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 1aa8cd5013b4..f2b5ea7e23ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -21,7 +21,7 @@ import java.io._ import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index c97ad4d72350..01901bbf85d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -31,7 +31,7 @@ import scala.util.Random import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ @@ -40,6 +40,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index c4c3283fb73f..ffdd635be4f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -17,8 +17,9 @@ package org.apache.spark.deploy.master -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 336cb24c19b1..1e8dabfbe6b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -20,8 +20,9 @@ package org.apache.spark.deploy.master import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatch, LeaderLatchListener} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.internal.Logging private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index b0cedef72e95..79f77212fefb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -25,8 +25,9 @@ import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index d7543926f385..ae16ce90c84b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.master.ui -import org.apache.spark.Logging import org.apache.spark.deploy.master.Master +import org.apache.spark.internal.Logging import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 7091513df07e..a057977eb0dd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -19,9 +19,10 @@ package org.apache.spark.deploy.mesos import java.util.concurrent.CountDownLatch -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.util.{ShutdownHookManager, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala index c0f9129a423f..6b297c4600a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala @@ -22,8 +22,9 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.ExternalShuffleService +import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler import org.apache.spark.network.shuffle.protocol.BlockTransferMessage diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index d3e092a34c17..c5a5876a896c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -30,7 +30,8 @@ import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import org.apache.spark.{Logging, SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 8e0862df4c29..14244ea5714c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -29,7 +29,8 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index ce02ee203a4b..a4efafcb27e5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -22,9 +22,9 @@ import java.io.{File, FileOutputStream, InputStream, IOException} import scala.collection.JavaConverters._ import scala.collection.Map -import org.apache.spark.Logging import org.apache.spark.SecurityManager import org.apache.spark.deploy.Command +import org.apache.spark.internal.Logging import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 7f4fe26c0d15..9c6bc5c62f25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -25,11 +25,12 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 208a1bb68edb..f9c92c3bb9f8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -24,9 +24,10 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c18c8c7c8603..1b7637a39ca7 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -29,12 +29,13 @@ import scala.concurrent.ExecutionContext import scala.util.{Failure, Random, Success} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index ab56fde938ba..af29de3b0896 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.worker -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 09ae64af8a42..6500cab73bf9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils import org.apache.spark.util.logging.RollingFileAppender diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index b45b6824949e..db696b04384b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils 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 3b5cb18da1b2..320a20033dae 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -28,6 +28,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 48372d70d52a..6327d55fe75c 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -29,6 +29,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler.{AccumulableInfo, DirectTaskResult, IndirectTaskResult, Task} diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index cfd9bcd65c56..680cfb733e9e 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -25,9 +25,10 @@ import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecuto import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.mesos.protobuf.ByteString -import org.apache.spark.{Logging, SparkConf, SparkEnv, TaskState} +import org.apache.spark.{SparkConf, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.cluster.mesos.MesosTaskLaunchData import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 9da9cb594058..02219a84abfd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} 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 bc98273add3a..978afaffab30 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Custom Input Format for reading and splitting flat binary files that contain records, diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/Logging.scala rename to core/src/main/scala/org/apache/spark/internal/Logging.scala index efab61e132a2..66a0cfec6296 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.internal import org.apache.log4j.{Level, LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 6841485f4b93..891facba3311 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -22,8 +22,9 @@ import java.io.IOException import org.apache.hadoop.mapreduce.{TaskAttemptContext => MapReduceTaskAttemptContext} import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} -import org.apache.spark.{Logging, SparkEnv, TaskContext} +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.executor.CommitDeniedException +import org.apache.spark.internal.Logging object SparkHadoopMapRedUtil extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index dbb0ad8d5c67..319718edb533 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -21,7 +21,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Implements policies and bookkeeping for sharing a adjustable-sized pool of memory between tasks. diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index e89b03e38b46..5e8abeecea9c 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -19,7 +19,8 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.BlockId import org.apache.spark.storage.memory.MemoryStore import org.apache.spark.unsafe.array.ByteArrayMethods diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 1d376adf1a12..6fcf26e3ecba 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -19,7 +19,7 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.BlockId import org.apache.spark.storage.memory.MemoryStore diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 8540984bfe82..979782ea40fd 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.matching.Regex -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils private[spark] class MetricsConfig(conf: SparkConf) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index e34cfc698dce..4da1017d282e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -25,7 +25,8 @@ import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 15d3540f3427..2de0f2033f2e 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -23,10 +23,10 @@ import java.nio.ByteBuffer import scala.concurrent.{Await, Future, Promise} import scala.concurrent.duration.Duration -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} -import org.apache.spark.storage.{BlockId, BlockManagerId, StorageLevel} +import org.apache.spark.storage.{BlockId, StorageLevel} private[spark] abstract class BlockTransferService extends ShuffleClient with Closeable with Logging { diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index e86933b94896..c1dbca5db200 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import scala.collection.JavaConverters._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 7de9df1e489f..c9ed12f4e1bd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import scala.reflect.ClassTag -import org.apache.spark.{ComplexFutureAction, FutureAction, JobSubmitter, Logging} +import org.apache.spark.{ComplexFutureAction, FutureAction, JobSubmitter} +import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils /** diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 0f579cfe420c..5e9230e7337c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,7 +17,8 @@ package org.apache.spark.rdd -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator import org.apache.spark.partial.PartialResult diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 805cd9fe1f63..08db96edd69b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -44,6 +44,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.Logging import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 8cbe80d65044..526138093d3e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -21,10 +21,11 @@ import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.internal.Logging import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala index dad90fc22084..56f53714cbe3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala @@ -19,7 +19,8 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, SparkEnv, TaskContext} +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index f23da39eb90d..fb9606ae388d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.Logging import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager} diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 2096a37de9dc..363004e587f2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -19,8 +19,9 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, Partitioner, RangePartitioner} +import org.apache.spark.{Partitioner, RangePartitioner} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -45,8 +46,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag, P <: Product2[K, V] : ClassTag] @DeveloperApi() ( self: RDD[P]) - extends Logging with Serializable -{ + extends Logging with Serializable { private val ordering = implicitly[Ordering[K]] /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 91460dc4067f..296179b75bc4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -41,6 +41,7 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} +import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 8782fcda162c..8a577c83e10d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala index 540cbd688b63..53d69ba26811 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -25,7 +25,8 @@ import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.google.common.base.Objects -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging /** * A general, named code block representing an operation that instantiates RDDs. diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index a9b3d52bbee0..fddb9353018a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.util.{SerializableConfiguration, Utils} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala index 92f625f7557e..74f187642af2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala @@ -22,6 +22,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path import org.apache.spark._ +import org.apache.spark.internal.Logging /** * An implementation of checkpointing that writes the RDD data to reliable storage. diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 92d9e3581ee5..1311b481c7c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala index 154398b57280..994e18676ec4 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -20,7 +20,8 @@ package org.apache.spark.rpc import scala.concurrent.Future import scala.reflect.ClassTag -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.RpcUtils /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 613d6ee7816d..4f8fe018b432 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import scala.concurrent.Promise import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc._ import org.apache.spark.util.ThreadUtils diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index 175463cc1031..fffbd5cd44a2 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -21,7 +21,8 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala index 6637e2321f67..7dd7e610a28e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala @@ -19,7 +19,7 @@ package org.apache.spark.rpc.netty import scala.concurrent.Promise -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc.{RpcAddress, RpcCallContext} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index c83a632e43fe..7f2192e1f5a7 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -29,7 +29,8 @@ import scala.reflect.ClassTag import scala.util.{DynamicVariable, Failure, Success, Try} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.network.TransportContext import org.apache.spark.network.client._ import org.apache.spark.network.netty.SparkTransportConf diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala index 9fd64e853575..56499c639f29 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala @@ -23,7 +23,8 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.rpc.{RpcAddress, RpcEnvStoppedException} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8a36af27bdd2..5cdc91316b69 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec import scala.collection.Map import scala.collection.mutable.{HashMap, HashSet, Stack} -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.existentials import scala.language.postfixOps @@ -36,6 +35,7 @@ import org.apache.commons.lang3.SerializationUtils 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.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD 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 2d76d08af6cd..a7d06391176d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -30,8 +30,9 @@ import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SPARK_VERSION, SparkConf} +import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index dfcdd113dfb9..0640f2605143 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -26,9 +26,9 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index ac8229a3c103..9012289f047c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.{Future, Promise} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 3e3ab15d8a24..2dd453cd6397 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} private sealed trait OutputCommitCoordinationMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 551e39a81b69..4cd13e2feaeb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index c6d957b65f3f..d32f5eb7bfe9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -24,7 +24,7 @@ import scala.io.Source import com.fasterxml.jackson.core.JsonParseException import org.json4s.jackson.JsonMethods._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.util.JsonProtocol /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 6c5827f75e63..5baebe8c1ff8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -22,7 +22,8 @@ import java.util.{NoSuchElementException, Properties} import scala.xml.XML -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 89207dd175ae..e30964a01bda 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -23,6 +23,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 0a45ef528332..586173f18036 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -25,9 +25,10 @@ import scala.collection.mutable import com.fasterxml.jackson.annotation.JsonTypeInfo -import org.apache.spark.{Logging, SparkConf, TaskEndReason} +import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.ui.SparkUI diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index c1c8b47128f2..a40b700cdd35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashSet import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index c94c4f55e9ce..7eb6d53c1095 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 36df032c2514..f7790fccc683 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -30,6 +30,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index f1339d530ad4..02764318f8a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -29,7 +29,7 @@ import scala.math.{max, min} import scala.util.control.NonFatal import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index d151de5f6a83..b7919efc4b12 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -22,7 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} +import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index d209645610c1..85d002011d64 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,9 +19,10 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.Semaphore -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} +import org.apache.spark.internal.Logging import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 092d9e418253..3971e6c3826c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -23,8 +23,9 @@ import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.zookeeper.KeeperException.NoNodeException -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index e79c543a9de2..374c79a7e5ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -20,7 +20,8 @@ package org.apache.spark.scheduler.cluster.mesos import org.apache.mesos.Protos.{ContainerInfo, Volume} import org.apache.mesos.Protos.ContainerInfo.DockerInfo -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging /** * A collection of utility functions which can be used by both the diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index a98f2f1fe5da..9a12a61f2ff9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -29,7 +29,8 @@ import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDrive import org.apache.mesos.Protos._ import org.apache.mesos.protobuf.{ByteString, GeneratedMessage} -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} +import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala index 5e7e6567a3e0..8370b61145e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.apache.mesos.protobuf.ByteString -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Wrapper for serializing the data sent when launching Mesos tasks. diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index c633d860ae6e..3473ef21b39a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,9 +21,10 @@ import java.io.File import java.net.URL import java.nio.ByteBuffer -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.internal.Logging import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 150ddc12e069..3d090a4353c3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -34,6 +34,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast +import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index e2951d8a3e09..8daca6c39063 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -25,7 +25,7 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[spark] object SerializationDebugger extends Logging { diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index dc182f596335..4edf3177396e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 2970968f0bd4..6cd7d6951851 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -21,8 +21,9 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 68aba52fd7c6..94d8c0d0fd3e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -21,7 +21,8 @@ import java.io._ import com.google.common.io.ByteStreams -import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 4f30da0878ee..6bb4ff94b546 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.hash import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.shuffle._ /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 7694e950be22..2cf76ec69c34 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle.hash import java.io.IOException import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle._ diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 9b1a27952842..97533bc089db 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle._ diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 4a7b1f07b6b6..8ab1cee2e842 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 22e21f0c62a2..c84022ddfeef 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -23,8 +23,9 @@ import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) private[v1] class EventLogDownloadResource( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index b23244ad5187..46fab7a89963 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -24,7 +24,8 @@ import scala.collection.mutable import com.google.common.collect.ConcurrentHashMultiset -import org.apache.spark.{Logging, SparkException, TaskContext} +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d21df4b95b3c..3bbdf48104c9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -30,6 +30,7 @@ import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.memory.MemoryManager import org.apache.spark.network._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 1cb027a3f543..c22d2e0fb61f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -21,7 +21,8 @@ import scala.collection.Iterable import scala.collection.generic.CanBuildFrom import scala.concurrent.Future -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{RpcUtils, ThreadUtils} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index fbb3df8c3e90..d2a5c69e15cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -23,8 +23,9 @@ import scala.collection.mutable import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index 9eca902f7454..d17ddbc16257 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -19,7 +19,8 @@ package org.apache.spark.storage import scala.concurrent.{ExecutionContext, Future} -import org.apache.spark.{Logging, MapOutputTracker, SparkEnv} +import org.apache.spark.{MapOutputTracker, SparkEnv} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index e51d96e57bc6..0666be2dcb01 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -20,8 +20,9 @@ package org.apache.spark.storage import java.io.{File, IOException} import java.util.UUID -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.internal.Logging import org.apache.spark.util.{ShutdownHookManager, Utils} /** diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 9cc408449773..ab97d2e4b8b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -20,8 +20,8 @@ package org.apache.spark.storage import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} import java.nio.channels.FileChannel -import org.apache.spark.Logging import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging import org.apache.spark.serializer.{SerializationStream, SerializerInstance} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index caecd97a0b72..5c28357ded6d 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -23,7 +23,8 @@ import java.nio.channels.FileChannel.MapMode import com.google.common.io.Closeables -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index b19c30e2ff77..25edb9f1e4c2 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -24,7 +24,8 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkException, TaskContext} +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} import org.apache.spark.shuffle.FetchFailedException diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 02d44dc73295..a7c1854a41ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -23,7 +23,8 @@ import java.util.LinkedHashMap import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, SparkConf, TaskContext} +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryManager import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel} import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index fa078ee25a12..2719e1ee98ba 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui import java.util.{Timer, TimerTask} import org.apache.spark._ +import org.apache.spark.internal.Logging /** * ConsoleProgressBar shows the progress of stages in the next line of the console. It poll the 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 6b3601250a65..31312fb064b1 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -34,7 +34,8 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SSLOptions} +import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 5324a7682960..60575225099f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,7 +21,8 @@ import java.util.{Date, ServiceLoader} import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, UIRoot} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 0493513d667c..aa2548a55412 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -25,7 +25,7 @@ import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.scope.RDDOperationGraph /** Utility functions for generating XML pages with spark content. */ diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 23d312525f2b..250b7f2e5f48 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -26,7 +26,8 @@ import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler import org.json4s.JsonAST.{JNothing, JValue} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SSLOptions} +import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} +import org.apache.spark.internal.Logging import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 2516b674fea8..ed3ab66e3b68 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index cb2827199853..bb6b663f1ead 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.{ListBuffer, StringBuilder} import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index e27d2e6c94f7..2f6924f7deef 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -24,7 +24,8 @@ import scala.collection.mutable.{Map, Set} import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.apache.xbean.asm5.Opcodes._ -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.Logging /** * A cleaner that renders closures serializable if they can be done so safely. diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index 542c5fccf458..153025cef247 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * An event loop to receive events from the caller and process all events in the event thread. It diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index 5e1fab009c7d..436c1951dee2 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * An event bus which posts events to its listeners. diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index 38523be791ce..bd26bfd848ff 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -24,7 +24,7 @@ import scala.util.Try import org.apache.hadoop.fs.FileSystem -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Various utility methods used by Spark. diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index a06db9a4fcfa..3f627a01453e 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -26,8 +26,8 @@ import scala.runtime.ScalaRunTime import com.google.common.collect.MapMaker -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.util.collection.OpenHashSet /** diff --git a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index 5e322557e964..95bf3f58bc77 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * The default uncaught exception handler for Executors terminates the whole process, to avoid diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 173302504106..32af0127bbf3 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 63b9d34b79fe..fe6063ce475a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -50,6 +50,7 @@ import org.slf4j.Logger import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ff9dad7d38bf..531f1c4dd276 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -26,9 +26,10 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import org.apache.spark.{Logging, SparkEnv, TaskContext} +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.storage.{BlockId, BlockManager} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 5afd6d6e22c6..2547d3da63a8 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -27,6 +27,7 @@ import com.google.common.io.ByteStreams import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.serializer._ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index e1592184ca6d..25ca2037bbac 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -17,7 +17,8 @@ package org.apache.spark.util.collection -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} /** diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 86bbaa20f6cf..fdb1495899bc 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.logging import java.io.{File, FileOutputStream, InputStream, IOException} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{IntParam, Utils} /** diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index d7b7219e179d..b34880d3a748 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.text.SimpleDateFormat import java.util.Calendar -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Defines the policy based on which [[org.apache.spark.util.logging.RollingFileAppender]] will diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index effe6fa2adcf..67822749112c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.apache.commons.math3.distribution.PoissonDistribution -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index e60678b30009..f98150536d8a 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.SpanSugar._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 9b6ab7b6bcfe..49c2bf6bcad1 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.scalatest.PrivateMethodTester +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.SparkDeploySchedulerBackend import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 99366a32c4e1..3228752b9638 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark // scalastyle:off import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} +import org.apache.spark.internal.Logging + /** * Base abstract class for all unit tests in Spark for handling common functionality. */ diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index b66aba91c57c..36273d722f50 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark import java.util.concurrent.Semaphore import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import org.apache.spark.internal.Logging + /** * Holds state shared across task threads in some ThreadingSuite tests. */ 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 bb2adff57e94..96cb4fd0ebee 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark._ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate +import org.apache.spark.internal.Logging import org.apache.spark.util.{ResetSystemProperties, Utils} // Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index 7017296bd136..7b46f9101d89 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, Master} import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index c874b95b0960..4ab000b53ad1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -36,7 +36,8 @@ import org.mockito.stubbing.Answer import org.scalatest.Matchers import org.scalatest.mock.MockitoSugar -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.status.api.v1.{ApplicationAttemptInfo => AttemptInfo, ApplicationInfo} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, ManualClock, Utils} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 5fd599e190c7..39c5857b1345 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -36,7 +36,8 @@ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.scheduler._ import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} 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 88b3a0e964ce..ddf73d637063 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,8 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.io.compress.{CompressionCodecFactory, GzipCodec} import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index a7de9cabe7cc..f9a7f151823a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.rdd import org.scalatest.Matchers -import org.apache.spark.{Logging, SharedSparkContext, SparkFunSuite} +import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers with Logging { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 43da6fc5b547..176d8930aad1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2afb595e6f10..a09a602d1368 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark._ +import org.apache.spark.internal.Logging class FakeSchedulerBackend extends SchedulerBackend { def start() {} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index d35ca411f408..167d3fd2e460 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.util.ManualClock class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 4d938d5c97de..280e4964980d 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -31,7 +31,8 @@ import org.mockito.ArgumentCaptor import org.mockito.Mockito.{atLeast, mock, verify} import org.scalatest.BeforeAndAfter -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.util.logging.{FileAppender, RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy} class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { 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 093d1bd6e594..4aa4854c36f3 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -34,7 +34,8 @@ import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { 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 fefa5165db19..65bf857e22c0 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,7 +20,8 @@ package org.apache.spark.util.collection import java.lang.{Float => JFloat, Integer => JInteger} import java.util.{Arrays, Comparator} -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.util.random.XORShiftRandom class SorterSuite extends SparkFunSuite with Logging { 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 39cb83d9eeb7..619e585b6ca1 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 @@ -24,6 +24,7 @@ import org.apache.spark._ import org.apache.spark.graphx._ import org.apache.spark.graphx.PartitionStrategy._ import org.apache.spark.graphx.lib._ +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel /** 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 d67da270a817..1d144db9864b 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 @@ -22,7 +22,8 @@ import java.io.{BufferedReader, InputStreamReader} import java.net.Socket import java.nio.charset.StandardCharsets -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.receiver.Receiver diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index 22a5654405dd..b00f32fb2524 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.streaming import org.apache.log4j.{Level, Logger} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** Utility functions for Spark Streaming examples. */ object StreamingExamples extends Logging { diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 0eaaf408c011..fb58ed789887 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -25,7 +25,8 @@ import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.Decoder -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 89d1811c9997..3713bda41b8e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -26,7 +26,7 @@ import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 475167aa70a2..d4881b140df3 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -27,7 +27,8 @@ import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties -import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index a5ea1d6d2848..d9d4240c056a 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -37,7 +37,8 @@ import kafka.utils.{ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.streaming.Time import org.apache.spark.util.Utils diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index a872781b78ee..39abe3c3e29d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -30,7 +30,8 @@ import kafka.serializer.Decoder import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} import org.apache.spark.util.ThreadUtils 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 b2c81d1534ee..f14ff6705fd9 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 @@ -32,7 +32,8 @@ import kafka.serializer.StringDecoder import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.Eventually -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index 6a73bc0e30d0..859fe9edb44f 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -29,7 +29,8 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionIn import com.amazonaws.services.kinesis.model.PutRecordRequest import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.streaming.dstream.DStream.toPairDStreamFunctions diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index 3996f168e69e..45dc3c388cb8 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -27,6 +27,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord import com.amazonaws.services.kinesis.model._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{BlockRDD, BlockRDDPartition} import org.apache.spark.storage.BlockId import org.apache.spark.util.NextIterator diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala index 1ca6d4302c2b..70b5cc7ca0e8 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala @@ -23,7 +23,7 @@ import scala.util.control.NonFatal import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason -import org.apache.spark.Logging +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} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 48ee2a959786..858368d135b6 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -28,11 +28,11 @@ import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration, Worker} import com.amazonaws.services.kinesis.model.Record +import org.apache.spark.internal.Logging import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.Duration import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} import org.apache.spark.util.Utils -import org.apache.spark.Logging private[kinesis] case class SerializableAWSCredentials(accessKeyId: String, secretKey: String) diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index 23b74da64237..41c6ab123bae 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -26,7 +26,7 @@ import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason import com.amazonaws.services.kinesis.model.Record -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.Duration /** diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 026387ed65d5..0fe66254e989 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -33,7 +33,7 @@ import com.amazonaws.services.dynamodbv2.document.DynamoDB import com.amazonaws.services.kinesis.AmazonKinesisClient import com.amazonaws.services.kinesis.model._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Shared utility methods for performing Kinesis tests that actually transfer data. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 1672f7d27c40..f678e5f1238f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -17,8 +17,9 @@ package org.apache.spark.graphx -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index efdc2481fe8e..d2e51d2ec443 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Implements a Pregel-like bulk-synchronous message-passing API. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 7c680dcb99cd..31373a53cf93 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -21,9 +21,9 @@ import scala.language.higherKinds import scala.language.implicitConversions import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.internal.Logging import org.apache.spark.util.collection.BitSet /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 9d9a26ebeb29..0a1622bca0f4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -20,8 +20,8 @@ package org.apache.spark.graphx.lib import scala.language.postfixOps import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. 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 ba67c66f86aa..80c6b6838faf 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 @@ -23,6 +23,7 @@ import scala.util._ import org.apache.spark._ import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** A collection of graph generating functions. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index f4c6214a5636..3a99979a886c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -26,8 +26,9 @@ import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.param.{Param, ParamMap, Params} import org.apache.spark.ml.util._ import org.apache.spark.sql.DataFrame diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 1f3325ad09ef..2538c0f477fc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml import scala.annotation.varargs -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.sql.DataFrame diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 82059b1d0ecb..5a8845fdb607 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.classification import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.regression.DecisionTreeRegressionModel diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 0d329d2c08d5..77e59d9188ef 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -23,8 +23,9 @@ import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN} import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 0de82b49ff6f..fe6a37fd6dc3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index e830d2a9adc4..e486e92c12aa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.feature import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param.{IntParam, _} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala index fe82324ab25c..a2b52835e177 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 55b751065664..7d21302f962b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index f3bc9f095a8c..091ca35283f1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -30,8 +30,9 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.{Logging, Partitioner} +import org.apache.spark.Partitioner import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 0901642d392d..ba5708ab8d9b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -23,8 +23,9 @@ import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 4cc2721aefb2..091e1d5fa854 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.regression import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.ml.tree.{DecisionTreeModel, GBTParams, TreeEnsembleModel, diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 46ba5589ff85..6e74cb54ad68 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -20,8 +20,9 @@ package org.apache.spark.ml.regression import breeze.stats.{distributions => dist} import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.optim._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 20a09982014c..fb733f9a349d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.regression import org.apache.hadoop.fs.Path -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index b4f17b8e2898..c8f3f70a9b44 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -24,8 +24,9 @@ import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, import breeze.stats.distributions.StudentsT import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.optim.WeightedLeastSquares import org.apache.spark.ml.PredictorParams diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index 44ab5b723bd7..b9acc6647293 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.tree.impl -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala index 172ea5282056..526b9c40628a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala @@ -23,8 +23,8 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.ml.tree.{LearningNode, Split} import org.apache.spark.mllib.tree.impl.BaggedPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index f994c258b2ca..91dc98569a21 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -22,7 +22,7 @@ import java.io.IOException import scala.collection.mutable import scala.util.Random -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 3eac616aeaf8..010e7d268657 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -22,8 +22,9 @@ import org.apache.hadoop.fs.Path import org.json4s.{DefaultFormats, JObject} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.classification.OneVsRestParams import org.apache.spark.ml.evaluation.Evaluator diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index 4f67e8c21994..4587e259e8bf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.tuning -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{DoubleParam, ParamMap, ParamValidators} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 329548f95a66..c95e536abd10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -24,8 +24,9 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.json4s.JsonDSL._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.param.{ParamPair, Params} import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 9026b97f1cbe..bf0d9d9231ac 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -24,8 +24,9 @@ import scala.collection.JavaConverters._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, SparseVector, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{Loader, Saveable} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index f0b9d64fc42f..64b838a1db97 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -22,9 +22,9 @@ import java.util.Random import scala.annotation.tailrec import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 9ccf96b9395b..01a0d31f14c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 67de62bc2e84..26f5600e6c07 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 81566b4779d6..fad808857a78 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index c9a96c68667a..adf20dc4b8b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.linalg.Vectors diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 9732dfa1744f..a422303dc933 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -21,10 +21,11 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index d99b89dc49eb..a8d7b8fdedb1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.clustering import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 319c54724dac..0a7a45b4f4e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.evaluation.binary._ import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.sql.DataFrame diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index c8ec0c16851f..c45742cebbfe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -22,9 +22,9 @@ import java.{lang => jl} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 0f4c97ec20c0..ef45c9fd9e5c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -18,10 +18,10 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 500187ae8630..5c35e1b91c9b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.feature -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index b046f684ca68..d3356b783fc2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -21,17 +21,16 @@ import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ArrayBuilder import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.Logging import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 790d6b101ee5..559241696422 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.fpm import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.AssociationRules.Rule import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index f5b4f2535f82..3f40af8f3ada 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -29,10 +29,11 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} -import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkContext, SparkException} +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.FPGrowth._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala index 3ea10779a183..659f875a6dc9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.fpm import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Calculate all patterns of a projected database in local mode. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index fd2f8d387556..4455681e5076 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -24,10 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala index 57ca4d3464f1..391f89aa1489 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala @@ -21,7 +21,8 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 27a73805678d..19cc942aba13 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * BLAS routines for MLlib's vectors and matrices. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 6f0b0a9bc600..89c332ae38fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -21,8 +21,9 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseMatrix => BDM, Matrix => BM} -import org.apache.spark.{Logging, Partitioner, SparkException} +import org.apache.spark.{Partitioner, SparkException} import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 0a36da410133..f6183a5eaadc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -25,8 +25,8 @@ import breeze.linalg.{axpy => brzAxpy, inv, svd => brzSvd, DenseMatrix => BDM, D MatrixSingularException, SparseVector => BSV} import breeze.numerics.{sqrt => brzSqrt} -import org.apache.spark.Logging import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 5873669b37e3..fbf657b0fac4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -21,8 +21,8 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{norm, DenseVector => BDV} -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index e8711651611d..82c2ce479005 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -22,8 +22,8 @@ import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index a7a843ab7689..c5b02d6b2e9c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.recommendation -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.{ALS => NewALS} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index c91729a9fd49..6f780b0da71f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -29,9 +29,10 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ import org.apache.spark.mllib.util.{Loader, Saveable} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index f7e3c5c501ae..4d5aaf14111c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,8 +17,9 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.feature.StandardScaler import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.optimization._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index 73948b2d9851..46deb545af3f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.regression import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala index 8a821d1b23ba..f131f6948ab1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat.correlation import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index fc684d7c5ffe..b760347bcb6f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat.correlation import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors} import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 4a3fb0646981..76ca6a8abd03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -22,7 +22,8 @@ import scala.collection.mutable import breeze.linalg.{DenseMatrix => BDM} import org.apache.commons.math3.distribution.ChiSquaredDistribution -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 2b3ed6df486c..baf9e5e7d1fd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -22,7 +22,7 @@ import scala.annotation.varargs import org.apache.commons.math3.distribution.{NormalDistribution, RealDistribution} import org.apache.commons.math3.stat.inference.{KolmogorovSmirnovTest => CommonMathKolmogorovSmirnovTest} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala index e990fe0768bc..4c382d7c2b79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.stat.test import scala.beans.BeanInfo -import org.apache.spark.Logging import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.StatCounter diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala index 911b4b923735..ff27f28459e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala @@ -26,7 +26,7 @@ import com.twitter.chill.MeatLocker import org.apache.commons.math3.stat.descriptive.StatisticalSummaryValues import org.apache.commons.math3.stat.inference.TTest -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.StatCounter diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 76c32208ea09..18f66e65f19c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -21,9 +21,9 @@ import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index d131f5da6c7e..eb40fb0391d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.tree -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index b7714b382a59..ec4c034169d0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -22,9 +22,9 @@ import java.io.IOException import scala.collection.mutable import scala.collection.JavaConverters._ -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 21ee49c45788..df13d291ca39 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.tree.impl import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index e007ee11054a..ea68ff64a80b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -23,9 +23,10 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index b373c2de3ea9..114a33a51d7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.FeatureType._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 59713c382e58..cbf49b6d5821 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -24,9 +24,10 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index dffe6e78939e..2c712d8f821a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD 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 d4f836ef33ad..1ffc62b38e85 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,8 @@ package org.apache.spark.ml.feature import scala.beans.{BeanInfo, BeanProperty} -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 2bedd70ce93e..dac76aa7a12c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -25,7 +25,8 @@ import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.linalg.Vectors @@ -33,7 +34,6 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.util.Utils class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { 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 832ccc0aacf8..e5567492a2c7 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 @@ -22,7 +22,8 @@ import scala.util.Random import breeze.linalg.{squaredDistance => breezeSquaredDistance, DenseMatrix => BDM} import org.json4s.jackson.JsonMethods.{parse => parseJson} -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.util.TestingUtils._ class VectorsSuite extends SparkFunSuite with Logging { 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 c3eeda012571..eaa819c2e6e3 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,7 +19,8 @@ package org.apache.spark.mllib.stat import breeze.linalg.{DenseMatrix => BDM, Matrix => BM} -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, SpearmanCorrelation} 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 acb3b953b53b..58828b3af9cb 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,8 @@ package org.apache.spark.mllib.tree -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala index 5fe5c8628973..7b4e14bb6aa4 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala @@ -17,9 +17,7 @@ package org.apache.spark.repl -import scala.collection.mutable.Set - -import org.apache.spark.Logging +import org.apache.spark.internal.Logging object Main extends Logging { diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index 5fb378112ef9..2b5d56a89590 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -13,7 +13,7 @@ import scala.tools.nsc.interpreter._ import scala.reflect.internal.util.BatchSourceFile import scala.tools.nsc.ast.parser.Tokens.EOF -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[repl] trait SparkExprTyper extends Logging { val repl: SparkIMain 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 2a8fa45e3c61..67a616dc15e5 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 @@ -10,13 +10,10 @@ package org.apache.spark.repl import java.net.URL -import org.apache.spark.annotation.DeveloperApi - import scala.reflect.io.AbstractFile import scala.tools.nsc._ import scala.tools.nsc.backend.JavaPlatform import scala.tools.nsc.interpreter._ - import scala.tools.nsc.interpreter.{Results => IR} import Predef.{println => _, _} import java.io.{BufferedReader, FileReader} @@ -42,9 +39,10 @@ import scala.tools.reflect.StdRuntimeTags._ import java.lang.{Class => jClass} import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} -import org.apache.spark.Logging import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils 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 fc260c031014..74a04d5a42bb 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 @@ -37,7 +37,8 @@ import scala.reflect.{ ClassTag, classTag } import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils import org.apache.spark.annotation.DeveloperApi diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala index f24d6da72437..1ba17dfd8e3d 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala @@ -7,8 +7,6 @@ package org.apache.spark.repl -import org.apache.spark.annotation.DeveloperApi - import scala.tools.nsc._ import scala.tools.nsc.interpreter._ @@ -16,7 +14,9 @@ import scala.tools.jline._ import scala.tools.jline.console.completer._ import Completion._ import scala.collection.mutable.ListBuffer -import org.apache.spark.Logging + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging /** * Represents an auto-completion tool for the supplied interpreter that diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index a58f4234da14..b822ff496c11 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -22,6 +22,7 @@ import java.io.File import scala.tools.nsc.GenericRunnerSettings import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils import org.apache.spark.sql.SQLContext diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 2bf1be1a582b..928aaa56293b 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -27,8 +27,9 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.xbean.asm5._ import org.apache.xbean.asm5.Opcodes._ -import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.util.{ParentClassLoader, Utils} /** diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index ce3f51bd72dd..9a143ee36ff4 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -40,6 +40,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.mock.MockitoSugar import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 72fe06545910..0d249a118cfa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 3dbe6348986b..2142671e605b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -24,7 +24,7 @@ import scala.language.existentials import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.ClassBodyEvaluator -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 5756f201fd60..908c32de4d89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.io.ObjectInputStream -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.StructType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 7f96db137ed2..51cfc5013045 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -21,7 +21,7 @@ import scala.annotation.tailrec import org.antlr.runtime._ import org.antlr.runtime.tree.CommonTree -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 56a3dd02f9ba..516b41cb138b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 7302b63646d6..62d54df98ecc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index c8aadb2ed534..0e02ad6057d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index 03414b2301e8..7eb72724d766 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 9ebacb4680dc..8e30349f50f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import com.google.common.util.concurrent.AtomicLongMap -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index ea6aa1850db4..3646c70ad2c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are 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 1751720a7db8..622a62abad89 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql import scala.language.implicitConversions -import org.apache.spark.Logging import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index ef85f1db895c..1b5a4999a8ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -21,9 +21,10 @@ import java.util.Properties import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, Partition} +import org.apache.spark.Partition import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index e4d930869298..4e3311f95808 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -25,9 +25,10 @@ import scala.collection.JavaConverters._ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalyst._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index d894825632a8..3a043dcc6af2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.reflect.runtime.universe.TypeTag import scala.util.Try -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.ScalaReflection diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 50f6562815c2..14b8b6fc3b38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import java.util.concurrent.locks.ReentrantReadWriteLock -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.storage.StorageLevel diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index e04683c499a3..a392b5341244 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -24,7 +24,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ -import org.apache.spark.{broadcast, Logging, SparkEnv} +import org.apache.spark.{broadcast, SparkEnv} +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SQLContext} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 38da82c47ce1..15627a70046c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.aggregate import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 5e1ea90cd4fc..213bca907b89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution.aggregate -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 1df38f7ff59c..9abae5357973 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.aggregate import scala.language.existentials -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, OuterScopes} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index ab178443dd12..f5776e7b8d49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.aggregate -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, MutableRow, _} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 738b9a35d1c9..d4e5db459ff5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator, UnsafeRowWriter} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala index b0e216feb559..63eae1b8685a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.columnar.{ColumnBuilder, NativeColumnBuilder} import org.apache.spark.sql.types.AtomicType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 44b07e461326..2abfd14916e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.command import java.util.NoSuchElementException -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Dataset, Row, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 3fb2e34101a6..07c89afafb6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e65a771202bc..6116cce17e75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -25,8 +25,8 @@ import scala.util.{Failure, Success, Try} import org.apache.hadoop.fs.Path -import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.execution.streaming.{FileStreamSource, Sink, Source} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 7f6671552ebd..4943702438d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{MapPartitionsRDD, RDD, UnionRDD} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 84e98c0f9e0f..62576ea4b2eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala index c4c7eccab6f6..e848f423eb11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SqlNewHadoopRDD.scala @@ -32,6 +32,7 @@ import org.apache.spark.{Partition => SparkPartition, _} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.SQLContext import org.apache.spark.sql.execution.datasources.parquet.UnsafeRowParquetRecordReader diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index d8aad5efe39d..c74ac8a282c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceF import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 6a0290c11228..e009a37f2de7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.StandardCharsets -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.datasources.CompressionCodecs private[sql] class CSVOptions( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 8c3f63d30732..7cf1b4c662ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets import com.univocity.parsers.csv.{CsvParser, CsvParserSettings, CsvWriter, CsvWriterSettings} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Read and parse CSV-like input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index eeb56f7c6c68..550101577520 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.RecordWriter import org.apache.hadoop.mapreduce.TaskAttemptContext import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala index 65af397451c5..7a6c0f9fed2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala @@ -21,7 +21,7 @@ import java.sql.{Driver, DriverManager} import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index 4dd3c50cdf03..6a5564addf48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -24,7 +24,8 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.StringUtils -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 64a820c6d741..b7ff5f72427a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.util.Try import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 3fa5ebf1bb81..6a20c666026c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -28,8 +28,8 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat -import org.apache.spark.Logging import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index e34309bab166..850e807b8677 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -28,7 +28,7 @@ import org.apache.parquet.io.api.RecordMaterializer import org.apache.parquet.schema._ import org.apache.parquet.schema.Type.Repetition -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index 8a128b4b6176..de6dd0fe3e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -29,7 +29,7 @@ import org.apache.parquet.schema.{GroupType, MessageType, PrimitiveType, Type} import org.apache.parquet.schema.OriginalType.{INT_32, LIST, UTF8} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.{BINARY, DOUBLE, FIXED_LEN_BYTE_ARRAY, INT32, INT64} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala index 0252c79d8e14..67bfd39697ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystWriteSupport.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext import org.apache.parquet.io.api.{Binary, RecordConsumer} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.catalyst.util.DateTimeUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 342034ca0ff9..3f0389beed85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -39,8 +39,9 @@ import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType import org.slf4j.bridge.SLF4JBridgeHandler -import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} +import org.apache.spark.{Partition => SparkPartition, SparkException} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow 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 034bf152620d..d5ce1243d914 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 @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution import scala.collection.mutable.HashSet -import org.apache.spark.{Accumulator, AccumulatorParam, Logging} +import org.apache.spark.{Accumulator, AccumulatorParam} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index 7f54ea97cdd6..fb60d68f986d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -22,7 +22,8 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, MapOutputStatistics, ShuffleDependency, SimpleFutureAction} +import org.apache.spark.{MapOutputStatistics, ShuffleDependency, SimpleFutureAction} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala index 0aa2785cb6b9..9c3145637d98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDF.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.python -import org.apache.spark.Logging import org.apache.spark.api.python.PythonFunction +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Expression, NonSQLExpression, Unevaluable} import org.apache.spark.sql.types.DataType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala index b912c5cc1dcf..bccd2a44d9fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.stat import scala.collection.mutable.{Map => MutableMap} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 71fd185b165e..0a0dccbad1cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.stat import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.expressions.{Cast, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 25c8a69b1f1e..787e93f54396 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.collection.OpenHashSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 4a0eb46b2212..0062b7fc75c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index d7ff44afadf2..a6504cd088b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -22,7 +22,8 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row, SQLContext} import org.apache.spark.sql.catalyst.encoders.{encoderFor, RowEncoder} import org.apache.spark.sql.catalyst.InternalRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 49646a99d68c..d3e823fdeb30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 9d3cd9bb14ec..4b4fa126b85f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 41762fc49c2c..5ae9e916adae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.execution.ui import scala.collection.mutable -import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} +import org.apache.spark.{JobExecutionStatus, SparkConf} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala index 4f50b2ecdc8f..e8675ce749a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.ui -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, SparkUITab} private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index cbdc37a2a162..9aabe2d0abe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -24,7 +24,7 @@ import scala.collection.immutable import org.apache.parquet.hadoop.ParquetOutputCommitter -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.parser.ParserConf import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 95ffc33011e8..6101b087029b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -25,9 +25,10 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index e6f8779929d5..3cae5355eecc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -22,8 +22,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable.ListBuffer import scala.util.control.NonFatal -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.QueryExecution /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index a7a826bc7a8d..fe12aa809909 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -21,7 +21,7 @@ import java.util.Random import org.scalatest.Matchers._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.functions.col import org.apache.spark.sql.test.SharedSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 9d7570fe7a85..052f4cbaebc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.columnar import java.nio.{ByteBuffer, ByteOrder} import java.nio.charset.StandardCharsets -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeProjection} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index f32ba5fe68a6..ee0d23a6e57c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -29,8 +29,9 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.{HiveServer2, HiveServerServerOptionsProcessor} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index d0d94da78fdc..a955314ba37a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -32,7 +32,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow} import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} 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 79cf7e8378f6..7fe31b002527 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 @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, CommandProces import org.apache.hadoop.hive.ql.session.SessionState import org.apache.thrift.transport.TSocket -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.HiveContext import org.apache.spark.util.ShutdownHookManager diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 0eb2627e7a58..b8bc8ea44dc8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index ca25d23c3e37..2594c5bfdb3a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -21,7 +21,8 @@ import java.io.PrintStream import scala.collection.JavaConverters._ -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.HiveContext import org.apache.spark.util.Utils diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 9954d3436d37..0c468a408ba9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -25,7 +25,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 3719da4925cc..c82fa4eaaa4e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState, SessionInfo} import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 27d1c8bab4d9..008108a5ce06 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState} import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 63e6bbcfee56..923ba8a30c5c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.hive.thriftserver.ui -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ import org.apache.spark.ui.{SparkUI, SparkUITab} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala index 439501fe1986..491f2aebb4f4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala @@ -22,7 +22,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.thrift.TException -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchItemException 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 05fc56958865..76b72a8ac5a2 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 @@ -38,8 +38,9 @@ import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.hadoop.util.VersionInfo -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index b6c78691e482..69bccfba4aa4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable, _} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} 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 00fc8af5781a..e802d3dfc386 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 @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 087b0c087c11..da910533d086 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeU import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector import org.apache.hadoop.io.Writable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.types.Decimal import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala index f3446a364b9f..cd417ce3cca9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 5754bbb82ddc..80b24dc9899f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -33,8 +33,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} -import org.apache.spark.Logging import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 3040ec93f8d8..845753488c86 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -34,7 +34,8 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} import org.apache.spark.sql.catalyst.catalog._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 70c10be25be9..4ecf866f9639 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorF import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{IntegralType, StringType} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 024f4dfeba9d..6a56bd96df3b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -27,8 +27,9 @@ import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmitUtils +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveContext import org.apache.spark.util.{MutableURLClassLoader, Utils} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 62e7c1223cd9..3566526561b2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -32,7 +32,8 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.io.Writable -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index abcc5656837d..efaa05237087 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, O ObjectInspectorFactory} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{analysis, InternalRow} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index fc0dfd2d2f81..a29d55ee25b2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 8240f2f2220c..019546694602 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -22,8 +22,8 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader} import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector -import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.HiveMetastoreTypes import org.apache.spark.sql.types.StructType diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala index 99a232f74fac..c025c12a90a2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument, SearchArgumentFactory} import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder import org.apache.hadoop.hive.serde2.io.DateWritable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.sources._ /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index ae041c5137f0..b5dc9106e236 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -31,8 +31,8 @@ import org.apache.hadoop.mapred.{InputFormat => MapRedInputFormat, JobConf, Outp import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat -import org.apache.spark.Logging import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow 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 c14c94f4c9c2..16747cab3747 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 @@ -30,6 +30,7 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.sql.{QueryTest, SQLContext} import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 16b2d042a248..cd96c85f3e20 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -22,7 +22,8 @@ import java.util.Collections import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 3d54da11ade4..74161d0a013c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -21,7 +21,8 @@ import java.io.File import org.apache.hadoop.util.VersionInfo -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal, NamedExpression} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 16c575bcc13a..f9f3d97ef3e4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -24,8 +24,9 @@ import java.util.concurrent.RejectedExecutionException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.streaming.scheduler.JobGenerator import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index d39768824593..54d736ee5101 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -21,7 +21,7 @@ import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} import org.apache.spark.streaming.scheduler.Job import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e7f3a213d468..3a664c4f5ca2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -35,6 +35,7 @@ import org.apache.spark._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.input.FixedLengthBinaryInputFormat +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger 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 102a0308185b..eb7b64eaf497 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 @@ -25,7 +25,8 @@ import scala.language.implicitConversions import scala.reflect.ClassTag import scala.util.matching.Regex -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{BlockRDD, PairRDDFunctions, RDD, RDDOperationScope} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index a9ce1131ce0c..365a6bc417a3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.Time import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index c6afac934c22..b2ec33e82dda 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -25,7 +25,7 @@ import java.util.concurrent.ArrayBlockingQueue import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index f7519c10c8eb..7853af562368 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 109af32cf4bb..e42bea6ec60d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -21,7 +21,8 @@ import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, SystemClock} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala index 6a1b672220bd..b2189103a027 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala @@ -19,7 +19,8 @@ package org.apache.spark.streaming.receiver import com.google.common.util.concurrent.{RateLimiter => GuavaRateLimiter} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging /** Provides waitToPush() method to limit the rate at which receivers consume data. * diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index e22e320b1712..4880884b0509 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -24,7 +24,8 @@ import scala.language.{existentials, postfixOps} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.storage._ import org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler._ import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 9cde5ae080c0..e0fe8d220658 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -24,7 +24,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent._ import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 8d4e6827d6a2..e41fd11963ba 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -27,7 +27,8 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.Time diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala index 92da0ced28fb..4f124a1356b5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.streaming.{StreamingContext, Time} /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index a83c0d922dce..86f069b0bd60 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -19,7 +19,8 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Success, Try} -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 3fed3d88354c..61f9e0974ca9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import scala.util.Failure -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{PairRDDFunctions, RDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.ui.UIUtils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 5ed5ae6358be..9c8e68b03db4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -27,7 +27,8 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.streaming.Time import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 6e24365c4e49..b3ae28700111 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -25,6 +25,7 @@ import scala.language.existentials import scala.util.{Failure, Success} import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorCacheTaskLocation, TaskLocation} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala index 84a3ca9d74e5..a73e6cc2cd9c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.scheduler.rate -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Implements a proportional-integral-derivative (PID) controller which acts on diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index b1bcd0680380..fa40436221d1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -23,7 +23,7 @@ import javax.servlet.http.HttpServletRequest import scala.collection.mutable.ArrayBuffer import scala.xml.{Node, Unparsed} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui._ import org.apache.spark.ui.{UIUtils => SparkUIUtils} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 0662c64a0ce9..c5f8aada3fc4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.ui -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.streaming.StreamingContext import org.apache.spark.ui.{SparkUI, SparkUITab} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 8cb45cdffa5d..165e81ea41a9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -27,7 +27,8 @@ import scala.concurrent.{Await, Promise} import scala.concurrent.duration._ import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.Utils diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index a3b7e783acd8..9b689f01b8d3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -29,7 +29,8 @@ import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{CompletionIterator, ThreadUtils} /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala index e79b139bdd03..14d9bc94a123 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * A reader for reading write ahead log files written using diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala index 5c3c7a6bf1b3..29cc1fa00ac0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit._ import scala.annotation.tailrec -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[streaming] class RateLimitedOutputStream(out: OutputStream, desiredBytesPerSec: Int) 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 e48eaf7913b1..9667af97f03b 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 @@ -23,7 +23,8 @@ import java.nio.ByteBuffer import scala.io.Source -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.IntParam 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 bfb53614050a..62e681e3e964 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 @@ -17,7 +17,7 @@ package org.apache.spark.streaming.util -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.util.{Clock, SystemClock} private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index ed616d8e810b..7542e2f5ecf2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -21,7 +21,8 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** A helper class with utility functions related to the WriteAheadLog interface */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 31e159e968c1..19ceb748e07f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.scalatest.BeforeAndAfter import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index cc2a67187e71..a2653000af55 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} 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 6406d53f8941..29bee4adf213 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -30,7 +30,7 @@ import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 95c1609d8e9a..2d509af85ae3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.memory.StaticMemoryManager import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.rpc.RpcEnv diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 898db85190b6..851013bb1e84 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -29,7 +29,8 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 2159edce2bf5..a80154e2fc81 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source import org.apache.spark.storage.StorageLevel diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9f96ee9b95f7..0f957a1b5570 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -29,8 +29,8 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.Logging import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 8269963edffa..fa975a146216 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -29,7 +29,8 @@ import org.scalatest.concurrent.Eventually.timeout import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.{Seconds => ScalaTestSeconds, Span} -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.{DStream, ForEachDStream, InputDStream} import org.apache.spark.streaming.scheduler._ diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 498471b23b51..a6a4fec3ba9e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -25,9 +25,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.ThreadUtils diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a06e677a0449..84445d60cd80 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} 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 f864ef13e0f1..6bbc8c2dfa19 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 @@ -49,9 +49,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} import org.apache.spark.util.Utils diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 4ef05c5a846d..869edf6c5b6a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** Client side methods to setup the Hadoop distributed cache */ private[spark] class ClientDistributedCacheManager() extends Logging { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala index 1ae278d76f02..3aa64071d478 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala @@ -24,9 +24,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class ExecutorDelegationTokenUpdater( diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 9cdbd6da6218..f956a4d1d595 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -37,8 +37,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.network.util.JavaUtils diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index e34cd8d1b710..d09430236285 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -32,9 +32,10 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 968f63527616..83d30b7352a0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -30,8 +30,9 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala index c4757e335b6c..4ed285230ff8 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala @@ -21,8 +21,9 @@ import java.util.concurrent.atomic.AtomicBoolean import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 20e2030fce08..9fc727904b1e 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -21,8 +21,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.yarn.api.records.YarnApplicationState -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} +import org.apache.spark.internal.Logging import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index ca26277b5061..0cc158b15a79 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -22,7 +22,8 @@ import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ From 2f9129a825e783385cfd556bb02fb724202b83b4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 17 Mar 2016 08:40:46 +0800 Subject: [PATCH 2/3] fix mima --- project/MimaExcludes.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ffc6fa0599c1..f473407c5408 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -537,6 +537,14 @@ object MimaExcludes { // SPARK-13927: add row/column iterator to local matrices ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.rowIter"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Matrix.colIter") + ) ++ Seq( + // [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Logging"), + (problem: Problem) => problem match { + case MissingTypesProblem(_, missing) + if missing.map(_.fullName).sameElements(Seq("org.apache.spark.Logging")) => false + case _ => true + } ) case v if v.startsWith("1.6") => Seq( From 0ee9d40aeffdca113c6fa717f950c0d972cdcf85 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 17 Mar 2016 12:44:10 +0800 Subject: [PATCH 3/3] more fix --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 3 ++- .../spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala | 3 ++- .../org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala | 1 + .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 1 + .../apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala | 1 + .../apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala | 3 ++- .../scheduler/cluster/ExtensionServiceIntegrationSuite.scala | 3 ++- 7 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 694bd97515b8..032965d0d9c2 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -29,7 +29,8 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.Utils diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index e89bb1c470d5..33af624cfda7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -40,7 +40,8 @@ import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 78b57da482f7..2f3a31cb046b 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -35,6 +35,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.launcher._ import org.apache.spark.util.Utils 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 8a92a7ecda54..26520529ecab 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 @@ -32,6 +32,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.launcher._ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 05c1e1613dd3..950ebd9a2d4d 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} import org.apache.spark.tags.ExtendedYarnTest diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 70b8732946a2..de14e36f4e95 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -32,8 +32,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.util.{ResetSystemProperties, Utils} class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging diff --git a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala index 338fbe2ef47f..6ea7984c6451 100644 --- a/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala @@ -19,8 +19,9 @@ package org.apache.spark.scheduler.cluster import org.scalatest.BeforeAndAfter -import org.apache.spark.{LocalSparkContext, Logging, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging /** * Test the integration with [[SchedulerExtensionServices]]