[CELEBORN-1565] Introduce warn-unused-import in Scala

### What changes were proposed in this pull request?
This PR aims to introduce `warn-unused-import` in Scala.

### Why are the changes needed?
There are currently many invalid imports, which can be checked using `-Ywarn-unused-import`.
And through `silencer`  plugin we can avoid some imports required in scala 2.11.

```scala
import org.apache.celeborn.common.util.FunctionConverter._
```

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
GA

Closes #2689 from cxzl25/CELEBORN-1565.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shaoyun Chen <csy@apache.org>
This commit is contained in:
sychen 2024-08-29 13:43:17 +08:00 committed by Shaoyun Chen
parent f801b7a32d
commit 3ee672e15d
No known key found for this signature in database
GPG Key ID: 81A87B8D54DB73A3
29 changed files with 22 additions and 47 deletions

View File

@ -41,7 +41,7 @@ import org.apache.celeborn.common.network.client.TransportClient
import org.apache.celeborn.common.network.protocol.TransportMessage import org.apache.celeborn.common.network.protocol.TransportMessage
import org.apache.celeborn.common.protocol.{MessageType, PartitionLocation, PbOpenStreamList, PbOpenStreamListResponse, PbStreamHandler} import org.apache.celeborn.common.protocol.{MessageType, PartitionLocation, PbOpenStreamList, PbOpenStreamListResponse, PbStreamHandler}
import org.apache.celeborn.common.protocol.message.StatusCode import org.apache.celeborn.common.protocol.message.StatusCode
import org.apache.celeborn.common.util.{ExceptionMaker, JavaUtils, ThreadUtils, Utils} import org.apache.celeborn.common.util.{JavaUtils, ThreadUtils, Utils}
class CelebornShuffleReader[K, C]( class CelebornShuffleReader[K, C](
handle: CelebornShuffleHandle[K, _, C], handle: CelebornShuffleHandle[K, _, C],

View File

@ -29,7 +29,7 @@ import org.apache.celeborn.common.meta.WorkerInfo
import org.apache.celeborn.common.protocol.PartitionLocation import org.apache.celeborn.common.protocol.PartitionLocation
import org.apache.celeborn.common.protocol.message.ControlMessages.WorkerResource import org.apache.celeborn.common.protocol.message.ControlMessages.WorkerResource
import org.apache.celeborn.common.protocol.message.StatusCode import org.apache.celeborn.common.protocol.message.StatusCode
import org.apache.celeborn.common.util.{JavaUtils, ThreadUtils, Utils} import org.apache.celeborn.common.util.{JavaUtils, ThreadUtils}
case class ChangePartitionRequest( case class ChangePartitionRequest(
context: RequestLocationCallContext, context: RequestLocationCallContext,

View File

@ -237,8 +237,6 @@ class LifecycleManager(val appUniqueId: String, val conf: CelebornConf) extends
} }
override def onStop(): Unit = { override def onStop(): Unit = {
import scala.concurrent.duration._
checkForShuffleRemoval.cancel(true) checkForShuffleRemoval.cancel(true)
ThreadUtils.shutdown(forwardMessageThread) ThreadUtils.shutdown(forwardMessageThread)

View File

@ -37,8 +37,8 @@ import org.apache.celeborn.common.meta.{ShufflePartitionLocationInfo, WorkerInfo
import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionType} import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionType}
import org.apache.celeborn.common.protocol.message.ControlMessages.{CommitFiles, CommitFilesResponse} import org.apache.celeborn.common.protocol.message.ControlMessages.{CommitFiles, CommitFilesResponse}
import org.apache.celeborn.common.protocol.message.StatusCode import org.apache.celeborn.common.protocol.message.StatusCode
import org.apache.celeborn.common.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.celeborn.common.rpc.RpcCallContext
import org.apache.celeborn.common.util.{CollectionUtils, JavaUtils, ThreadUtils, Utils} import org.apache.celeborn.common.util.{CollectionUtils, JavaUtils, Utils}
// Can Remove this if celeborn don't support scala211 in future // Can Remove this if celeborn don't support scala211 in future
import org.apache.celeborn.common.util.FunctionConverter._ import org.apache.celeborn.common.util.FunctionConverter._
import org.apache.celeborn.common.util.ThreadUtils.awaitResult import org.apache.celeborn.common.util.ThreadUtils.awaitResult

View File

@ -33,7 +33,7 @@ import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.internal.config._ import org.apache.celeborn.common.internal.config._
import org.apache.celeborn.common.network.util.ByteUnit import org.apache.celeborn.common.network.util.ByteUnit
import org.apache.celeborn.common.protocol._ import org.apache.celeborn.common.protocol._
import org.apache.celeborn.common.protocol.StorageInfo.{typesMap, validate, Type} import org.apache.celeborn.common.protocol.StorageInfo.Type
import org.apache.celeborn.common.protocol.StorageInfo.Type.{HDD, SSD} import org.apache.celeborn.common.protocol.StorageInfo.Type.{HDD, SSD}
import org.apache.celeborn.common.rpc.RpcTimeout import org.apache.celeborn.common.rpc.RpcTimeout
import org.apache.celeborn.common.util.{JavaUtils, Utils} import org.apache.celeborn.common.util.{JavaUtils, Utils}

View File

@ -17,9 +17,6 @@
package org.apache.celeborn.common.client package org.apache.celeborn.common.client
import java.util
import scala.collection.JavaConverters._
import scala.util.Random import scala.util.Random
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf

View File

@ -19,7 +19,6 @@ package org.apache.celeborn.common.meta
import java.io.File import java.io.File
import java.util import java.util
import java.util.function.BiFunction
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer

View File

@ -38,7 +38,7 @@ import org.apache.celeborn.common.network.protocol.{RequestMessage => NRequestMe
import org.apache.celeborn.common.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} import org.apache.celeborn.common.network.sasl.{SaslClientBootstrap, SaslServerBootstrap}
import org.apache.celeborn.common.network.sasl.registration.{RegistrationClientBootstrap, RegistrationServerBootstrap} import org.apache.celeborn.common.network.sasl.registration.{RegistrationClientBootstrap, RegistrationServerBootstrap}
import org.apache.celeborn.common.network.server._ import org.apache.celeborn.common.network.server._
import org.apache.celeborn.common.protocol.{RpcNameConstants, TransportModuleConstants} import org.apache.celeborn.common.protocol.RpcNameConstants
import org.apache.celeborn.common.rpc._ import org.apache.celeborn.common.rpc._
import org.apache.celeborn.common.serializer.{JavaSerializer, JavaSerializerInstance, SerializationStream} import org.apache.celeborn.common.serializer.{JavaSerializer, JavaSerializerInstance, SerializationStream}
import org.apache.celeborn.common.util.{ByteBufferInputStream, ByteBufferOutputStream, JavaUtils, ThreadUtils, Utils} import org.apache.celeborn.common.util.{ByteBufferInputStream, ByteBufferOutputStream, JavaUtils, ThreadUtils, Utils}

View File

@ -20,8 +20,6 @@ package org.apache.celeborn.common.util
import java.io.{File, IOException} import java.io.{File, IOException}
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.JavaConverters._
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.security.UserGroupInformation

View File

@ -23,10 +23,9 @@ import java.util.concurrent.locks.ReentrantLock
import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future}
import scala.concurrent.duration.{Duration, DurationInt, FiniteDuration} import scala.concurrent.duration.{Duration, DurationInt, FiniteDuration}
import scala.language.higherKinds
import scala.util.control.NonFatal import scala.util.control.NonFatal
import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.celeborn.common.exception.CelebornException import org.apache.celeborn.common.exception.CelebornException
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging

View File

@ -21,7 +21,6 @@ import org.apache.celeborn.CelebornFunSuite
import org.apache.celeborn.common.CelebornConf._ import org.apache.celeborn.common.CelebornConf._
import org.apache.celeborn.common.internal.config.ConfigEntry import org.apache.celeborn.common.internal.config.ConfigEntry
import org.apache.celeborn.common.protocol.StorageInfo import org.apache.celeborn.common.protocol.StorageInfo
import org.apache.celeborn.common.util.Utils
class CelebornConfSuite extends CelebornFunSuite { class CelebornConfSuite extends CelebornFunSuite {

View File

@ -17,10 +17,6 @@
package org.apache.celeborn.common.client package org.apache.celeborn.common.client
import java.util
import scala.collection.JavaConverters._
import org.apache.celeborn.CelebornFunSuite import org.apache.celeborn.CelebornFunSuite
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.protocol.RpcNameConstants import org.apache.celeborn.common.protocol.RpcNameConstants

View File

@ -29,7 +29,6 @@ import org.apache.celeborn.common.exception.CelebornException
import org.apache.celeborn.common.network.client.TransportClient import org.apache.celeborn.common.network.client.TransportClient
import org.apache.celeborn.common.protocol.TransportModuleConstants import org.apache.celeborn.common.protocol.TransportModuleConstants
import org.apache.celeborn.common.rpc._ import org.apache.celeborn.common.rpc._
import org.apache.celeborn.common.util.ThreadUtils
class NettyRpcEnvSuite extends RpcEnvSuite with TimeLimits { class NettyRpcEnvSuite extends RpcEnvSuite with TimeLimits {

View File

@ -20,7 +20,6 @@ package org.apache.celeborn.common.rpc.netty
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.network.TestHelper import org.apache.celeborn.common.network.TestHelper
import org.apache.celeborn.common.network.ssl.SslSampleConfigs import org.apache.celeborn.common.network.ssl.SslSampleConfigs
import org.apache.celeborn.common.protocol.TransportModuleConstants
class SSLNettyRpcEnvSuite extends NettyRpcEnvSuite { class SSLNettyRpcEnvSuite extends NettyRpcEnvSuite {

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.shaded.org.apache.commons.lang3.RandomStringUtils
import org.apache.celeborn.CelebornFunSuite import org.apache.celeborn.CelebornFunSuite
import org.apache.celeborn.common.identity.UserIdentifier import org.apache.celeborn.common.identity.UserIdentifier
import org.apache.celeborn.common.meta.{ApplicationMeta, DeviceInfo, DiskFileInfo, DiskInfo, FileInfo, ReduceFileMeta, WorkerEventInfo, WorkerInfo, WorkerStatus} import org.apache.celeborn.common.meta.{ApplicationMeta, DeviceInfo, DiskFileInfo, DiskInfo, ReduceFileMeta, WorkerEventInfo, WorkerInfo, WorkerStatus}
import org.apache.celeborn.common.protocol.{PartitionLocation, PbPackedWorkerResource, PbWorkerResource, StorageInfo} import org.apache.celeborn.common.protocol.{PartitionLocation, PbPackedWorkerResource, PbWorkerResource, StorageInfo}
import org.apache.celeborn.common.protocol.message.ControlMessages.WorkerResource import org.apache.celeborn.common.protocol.message.ControlMessages.WorkerResource
import org.apache.celeborn.common.quota.ResourceConsumption import org.apache.celeborn.common.quota.ResourceConsumption

View File

@ -20,7 +20,7 @@ package org.apache.celeborn.service.deploy.master
import java.io.IOException import java.io.IOException
import java.net.BindException import java.net.BindException
import java.util import java.util
import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledFuture, TimeUnit} import java.util.concurrent.{ExecutorService, ScheduledFuture, TimeUnit}
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.function.ToLongFunction import java.util.function.ToLongFunction
@ -43,12 +43,12 @@ import org.apache.celeborn.common.metrics.source.{JVMCPUSource, JVMSource, Resou
import org.apache.celeborn.common.network.CelebornRackResolver import org.apache.celeborn.common.network.CelebornRackResolver
import org.apache.celeborn.common.network.protocol.TransportMessage import org.apache.celeborn.common.network.protocol.TransportMessage
import org.apache.celeborn.common.protocol._ import org.apache.celeborn.common.protocol._
import org.apache.celeborn.common.protocol.message.{ControlMessages, StatusCode}
import org.apache.celeborn.common.protocol.message.ControlMessages._ import org.apache.celeborn.common.protocol.message.ControlMessages._
import org.apache.celeborn.common.protocol.message.StatusCode
import org.apache.celeborn.common.quota.ResourceConsumption import org.apache.celeborn.common.quota.ResourceConsumption
import org.apache.celeborn.common.rpc._ import org.apache.celeborn.common.rpc._
import org.apache.celeborn.common.rpc.{RpcSecurityContextBuilder, ServerSaslContextBuilder} import org.apache.celeborn.common.rpc.{RpcSecurityContextBuilder, ServerSaslContextBuilder}
import org.apache.celeborn.common.util.{CelebornHadoopUtils, CollectionUtils, JavaUtils, PbSerDeUtils, SignalUtils, ThreadUtils, Utils} import org.apache.celeborn.common.util.{CelebornHadoopUtils, JavaUtils, PbSerDeUtils, SignalUtils, ThreadUtils, Utils}
import org.apache.celeborn.server.common.{HttpService, Service} import org.apache.celeborn.server.common.{HttpService, Service}
import org.apache.celeborn.service.deploy.master.clustermeta.SingleMasterMetaManager import org.apache.celeborn.service.deploy.master.clustermeta.SingleMasterMetaManager
import org.apache.celeborn.service.deploy.master.clustermeta.ha.{HAHelper, HAMasterMetaManager, MetaHandler} import org.apache.celeborn.service.deploy.master.clustermeta.ha.{HAHelper, HAMasterMetaManager, MetaHandler}

View File

@ -19,14 +19,13 @@ package org.apache.celeborn.service.deploy.master
import java.nio.file.Files import java.nio.file.Files
import org.mockito.Mockito.{mock, times, verify} import org.mockito.Mockito.mock
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
import org.scalatest.funsuite.AnyFunSuite import org.scalatest.funsuite.AnyFunSuite
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.protocol.{PbCheckForWorkerTimeout, PbRegisterWorker} import org.apache.celeborn.common.protocol.{PbCheckForWorkerTimeout, PbRegisterWorker}
import org.apache.celeborn.common.protocol.message.ControlMessages.{ApplicationLost, HeartbeatFromApplication}
import org.apache.celeborn.common.util.{CelebornExitKind, Utils} import org.apache.celeborn.common.util.{CelebornExitKind, Utils}
class MasterSuite extends AnyFunSuite class MasterSuite extends AnyFunSuite

View File

@ -958,12 +958,14 @@
<version>${maven.plugin.scala.version}</version> <version>${maven.plugin.scala.version}</version>
<configuration> <configuration>
<args> <args>
<arg>-Ywarn-unused-import</arg>
<arg>-unchecked</arg> <arg>-unchecked</arg>
<arg>-deprecation</arg> <arg>-deprecation</arg>
<arg>-feature</arg> <arg>-feature</arg>
<arg>-explaintypes</arg> <arg>-explaintypes</arg>
<arg>-Xfatal-warnings</arg> <arg>-Xfatal-warnings</arg>
<arg>-P:silencer:globalFilters=.*deprecated.*</arg> <arg>-P:silencer:globalFilters=.*deprecated.*</arg>
<arg>-P:silencer:lineContentFilters=.*FunctionConverter.*</arg>
</args> </args>
<compilerPlugins> <compilerPlugins>
<compilerPlugin> <compilerPlugin>

View File

@ -27,7 +27,7 @@ import scala.collection.mutable
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.authentication.HttpAuthSchemes import org.apache.celeborn.common.authentication.HttpAuthSchemes
import org.apache.celeborn.common.authentication.HttpAuthSchemes.{HttpAuthScheme, _} import org.apache.celeborn.common.authentication.HttpAuthSchemes.HttpAuthScheme
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.server.common.Service import org.apache.celeborn.server.common.Service
import org.apache.celeborn.server.common.http.HttpAuthUtils.AUTHORIZATION_HEADER import org.apache.celeborn.server.common.http.HttpAuthUtils.AUTHORIZATION_HEADER

View File

@ -24,7 +24,6 @@ import org.scalatest.concurrent.Waiters.{interval, timeout}
import org.scalatest.time.SpanSugar.convertIntToGrainOfTime import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
import org.apache.celeborn.CelebornFunSuite import org.apache.celeborn.CelebornFunSuite
import org.apache.celeborn.client.WithShuffleClientSuite
// TODO need add shuffle client test // TODO need add shuffle client test
class DeploySuite extends CelebornFunSuite with WithMiniKube { class DeploySuite extends CelebornFunSuite with WithMiniKube {

View File

@ -26,9 +26,8 @@ import org.apache.celeborn.client.{LifecycleManager, WithShuffleClientSuite}
import org.apache.celeborn.client.LifecycleManager.ShuffleFailedWorkers import org.apache.celeborn.client.LifecycleManager.ShuffleFailedWorkers
import org.apache.celeborn.client.commit.CommitFilesParam import org.apache.celeborn.client.commit.CommitFilesParam
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.protocol.message.ControlMessages.WorkerResource
import org.apache.celeborn.common.protocol.message.StatusCode import org.apache.celeborn.common.protocol.message.StatusCode
import org.apache.celeborn.common.util.{CelebornExitKind, Utils} import org.apache.celeborn.common.util.Utils
import org.apache.celeborn.service.deploy.MiniClusterFeature import org.apache.celeborn.service.deploy.MiniClusterFeature
import org.apache.celeborn.service.deploy.worker.CommitInfo import org.apache.celeborn.service.deploy.worker.CommitInfo

View File

@ -19,8 +19,6 @@ package org.apache.celeborn.tests.client
import java.util import java.util
import scala.collection.JavaConverters._
import org.apache.celeborn.client.{LifecycleManager, WithShuffleClientSuite} import org.apache.celeborn.client.{LifecycleManager, WithShuffleClientSuite}
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.protocol.message.StatusCode import org.apache.celeborn.common.protocol.message.StatusCode

View File

@ -31,7 +31,7 @@ import org.roaringbitmap.RoaringBitmap
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.identity.UserIdentifier import org.apache.celeborn.common.identity.UserIdentifier
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.meta.{ReduceFileMeta, WorkerInfo, WorkerPartitionLocationInfo} import org.apache.celeborn.common.meta.{WorkerInfo, WorkerPartitionLocationInfo}
import org.apache.celeborn.common.metrics.MetricsSystem import org.apache.celeborn.common.metrics.MetricsSystem
import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionSplitMode, PartitionType, StorageInfo} import org.apache.celeborn.common.protocol.{PartitionLocation, PartitionSplitMode, PartitionType, StorageInfo}
import org.apache.celeborn.common.protocol.message.ControlMessages._ import org.apache.celeborn.common.protocol.message.ControlMessages._

View File

@ -20,12 +20,9 @@ package org.apache.celeborn.service.deploy.worker
import java.io.{FileNotFoundException, IOException} import java.io.{FileNotFoundException, IOException}
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import java.util import java.util
import java.util.concurrent.{Future => JFuture}
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.function.Consumer import java.util.function.Consumer
import scala.collection.JavaConverters._
import com.google.common.base.Throwables import com.google.common.base.Throwables
import com.google.protobuf.GeneratedMessageV3 import com.google.protobuf.GeneratedMessageV3
import io.netty.util.concurrent.{Future, GenericFutureListener} import io.netty.util.concurrent.{Future, GenericFutureListener}

View File

@ -26,13 +26,12 @@ import scala.concurrent.{Await, Promise}
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.{Failure, Success, Try} import scala.util.{Failure, Success, Try}
import com.google.common.base.Throwables
import com.google.protobuf.GeneratedMessageV3 import com.google.protobuf.GeneratedMessageV3
import io.netty.buffer.ByteBuf import io.netty.buffer.ByteBuf
import org.apache.celeborn.common.exception.{AlreadyClosedException, CelebornIOException} import org.apache.celeborn.common.exception.{AlreadyClosedException, CelebornIOException}
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.meta.{DiskFileInfo, DiskStatus, WorkerInfo, WorkerPartitionLocationInfo} import org.apache.celeborn.common.meta.{DiskStatus, WorkerInfo, WorkerPartitionLocationInfo}
import org.apache.celeborn.common.metrics.source.Source import org.apache.celeborn.common.metrics.source.Source
import org.apache.celeborn.common.network.buffer.{NettyManagedBuffer, NioManagedBuffer} import org.apache.celeborn.common.network.buffer.{NettyManagedBuffer, NioManagedBuffer}
import org.apache.celeborn.common.network.client.{RpcResponseCallback, TransportClient, TransportClientFactory} import org.apache.celeborn.common.network.client.{RpcResponseCallback, TransportClient, TransportClientFactory}

View File

@ -27,8 +27,8 @@ import com.google.common.collect.Sets
import org.apache.celeborn.common.CelebornConf import org.apache.celeborn.common.CelebornConf
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.meta.WorkerStatus import org.apache.celeborn.common.meta.WorkerStatus
import org.apache.celeborn.common.protocol.{PbWorkerStatus, WorkerEventType}
import org.apache.celeborn.common.protocol.PbWorkerStatus.State import org.apache.celeborn.common.protocol.PbWorkerStatus.State
import org.apache.celeborn.common.protocol.WorkerEventType
import org.apache.celeborn.service.deploy.worker.storage.StorageManager import org.apache.celeborn.service.deploy.worker.storage.StorageManager
private[celeborn] class WorkerStatusManager(conf: CelebornConf) extends Logging { private[celeborn] class WorkerStatusManager(conf: CelebornConf) extends Logging {

View File

@ -18,14 +18,12 @@
package org.apache.celeborn.service.deploy.worker.storage package org.apache.celeborn.service.deploy.worker.storage
import java.io.IOException import java.io.IOException
import java.nio.channels.ClosedByInterruptException
import java.util.concurrent.{ExecutorService, LinkedBlockingQueue, TimeUnit} import java.util.concurrent.{ExecutorService, LinkedBlockingQueue, TimeUnit}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLongArray} import java.util.concurrent.atomic.{AtomicBoolean, AtomicLongArray}
import scala.collection.JavaConverters._
import scala.util.Random import scala.util.Random
import io.netty.buffer.{CompositeByteBuf, PooledByteBufAllocator, Unpooled} import io.netty.buffer.{CompositeByteBuf, PooledByteBufAllocator}
import org.apache.celeborn.common.internal.Logging import org.apache.celeborn.common.internal.Logging
import org.apache.celeborn.common.meta.{DiskStatus, TimeWindow} import org.apache.celeborn.common.meta.{DiskStatus, TimeWindow}

View File

@ -21,7 +21,7 @@ import java.io.IOException
import java.net.BindException import java.net.BindException
import java.nio.file.Files import java.nio.file.Files
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.locks.{Lock, ReentrantLock} import java.util.concurrent.locks.ReentrantLock
import scala.collection.mutable import scala.collection.mutable

View File

@ -19,7 +19,7 @@ package org.apache.celeborn.service.deploy.worker.storage
import org.apache.celeborn.common.util.CelebornExitKind import org.apache.celeborn.common.util.CelebornExitKind
import org.apache.celeborn.server.common.HttpService import org.apache.celeborn.server.common.HttpService
import org.apache.celeborn.server.common.http.{ApiBaseResourceAuthenticationSuite, ApiBaseResourceSuite} import org.apache.celeborn.server.common.http.ApiBaseResourceAuthenticationSuite
import org.apache.celeborn.service.deploy.worker.{Worker, WorkerArguments} import org.apache.celeborn.service.deploy.worker.{Worker, WorkerArguments}
class ApiWorkerResourceAuthenticationSuite extends ApiBaseResourceAuthenticationSuite { class ApiWorkerResourceAuthenticationSuite extends ApiBaseResourceAuthenticationSuite {