Skip to content

Commit bc67612

Browse files
laglangyuejiyong-lee-dev
authored andcommitted
[SPARK-45629][CORE][SQL][CONNECT][ML][STREAMING][BUILD][EXAMPLES] Fix Implicit definition should have explicit type
### What changes were proposed in this pull request? This PR aims to fix `Implicit definition should have explicit type` in Scala 2.13. This pr includes: 1. Declaration types for global variables of implicit 2. Add scala.annotation.warn ### Why are the changes needed? - For implicit global variables without explicit type declaration, will get warnning : warning: Implicit definition should have explicit type (inferred String) [quickfixable] - No modifications are required for local variables. Additionally, to handle cases involving reflection-related types like ClassTag in implicit variables, the [scala.annotation.warn](https://github.com/scala.annotation.warn) annotation is used to suppress the warning. Furthermore, warnings generated in Spark will be treated as errors: [error] ... Implicit definition should have explicit type (inferred org.json4s.DefaultFormats.type) [quickfixable] ... [error] implicit val formats = org.json4s.DefaultFormats Jira link: SPARK-45314: https://issues.apache.org/jira/browse/SPARK-45629 Related issue link about `implicit` : scala/bug#5265 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Most of the testing is completed through CI, and the example module is locally compiled and tested in IDEA Additionally, there are some writing changes that are verified through demo code ### Was this patch authored or co-authored using generative AI tooling? no Closes apache#43526 from laglangyue/SPARK-45629. Lead-authored-by: tangjiafu <jiafu.tang@qq.com> Co-authored-by: tangjiafu <tangjiafu@corp.netease.com> Signed-off-by: yangjie01 <yangjie01@baidu.com> # Conflicts: # connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala # core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala # pom.xml # project/SparkBuild.scala # sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala # sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadata.scala
1 parent b8f486f commit bc67612

File tree

78 files changed

+906
-455
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

78 files changed

+906
-455
lines changed

connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala

Lines changed: 303 additions & 52 deletions
Large diffs are not rendered by default.

connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -21,14 +21,14 @@ import scala.collection.mutable.HashMap
2121
import scala.util.control.NonFatal
2222

2323
import org.apache.kafka.common.TopicPartition
24-
import org.json4s.NoTypeHints
24+
import org.json4s.{Formats, NoTypeHints}
2525
import org.json4s.jackson.Serialization
2626

2727
/**
2828
* Utilities for converting Kafka related objects to and from json.
2929
*/
3030
private object JsonUtils {
31-
private implicit val formats = Serialization.formats(NoTypeHints)
31+
private implicit val formats: Formats = Serialization.formats(NoTypeHints)
3232

3333
/**
3434
* Read TopicPartitions from json string
@@ -96,10 +96,8 @@ private object JsonUtils {
9696
*/
9797
def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = {
9898
val result = new HashMap[String, HashMap[Int, Long]]()
99-
implicit val order = new Ordering[TopicPartition] {
100-
override def compare(x: TopicPartition, y: TopicPartition): Int = {
101-
Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition))
102-
}
99+
implicit val order: Ordering[TopicPartition] = (x: TopicPartition, y: TopicPartition) => {
100+
Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition))
103101
}
104102
val partitions = partitionOffsets.keySet.toSeq.sorted // sort for more determinism
105103
partitions.foreach { tp =>

core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import scala.concurrent.ExecutionContext.Implicits.global
3030
import scala.concurrent.duration._
3131
import scala.sys.process._
3232

33+
import org.json4s.Formats
3334
import org.json4s.jackson.JsonMethods
3435

3536
import org.apache.spark.{SparkConf, SparkContext}
@@ -340,7 +341,7 @@ private object FaultToleranceTest extends App with Logging {
340341
private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File)
341342
extends Logging {
342343

343-
implicit val formats = org.json4s.DefaultFormats
344+
implicit val formats: Formats = org.json4s.DefaultFormats
344345
var state: RecoveryState.Value = _
345346
var liveWorkerIPs: List[String] = _
346347
var numLiveApps = 0
@@ -383,7 +384,7 @@ private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile
383384
private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File)
384385
extends Logging {
385386

386-
implicit val formats = org.json4s.DefaultFormats
387+
implicit val formats: Formats = org.json4s.DefaultFormats
387388

388389
logDebug("Created worker: " + this)
389390

core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import java.nio.file.Files
2323
import scala.collection.mutable
2424
import scala.util.control.NonFatal
2525

26-
import org.json4s.{DefaultFormats, Extraction}
26+
import org.json4s.{DefaultFormats, Extraction, Formats}
2727
import org.json4s.jackson.JsonMethods.{compact, render}
2828

2929
import org.apache.spark.SparkException
@@ -114,7 +114,7 @@ private[spark] object StandaloneResourceUtils extends Logging {
114114
private def writeResourceAllocationJson[T](
115115
allocations: Seq[T],
116116
jsonFile: File): Unit = {
117-
implicit val formats = DefaultFormats
117+
implicit val formats: Formats = DefaultFormats
118118
val allocationJson = Extraction.decompose(allocations)
119119
Files.write(jsonFile.toPath, compact(render(allocationJson)).getBytes())
120120
}

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@ import scala.util.{Failure, Success}
2727
import scala.util.control.NonFatal
2828

2929
import io.netty.util.internal.PlatformDependent
30-
import org.json4s.DefaultFormats
3130

3231
import org.apache.spark._
3332
import org.apache.spark.TaskState.TaskState
@@ -60,8 +59,6 @@ private[spark] class CoarseGrainedExecutorBackend(
6059

6160
import CoarseGrainedExecutorBackend._
6261

63-
private implicit val formats = DefaultFormats
64-
6562
private[spark] val stopping = new AtomicBoolean(false)
6663
var executor: Executor = null
6764
@volatile var driver: Option[RpcEndpointRef] = None

core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.resource
1919

2020
import scala.util.control.NonFatal
2121

22-
import org.json4s.{DefaultFormats, Extraction, JValue}
22+
import org.json4s.{DefaultFormats, Extraction, Formats, JValue}
2323
import org.json4s.jackson.JsonMethods._
2424

2525
import org.apache.spark.SparkException
@@ -69,7 +69,7 @@ private[spark] object ResourceInformation {
6969
* Parses a JSON string into a [[ResourceInformation]] instance.
7070
*/
7171
def parseJson(json: String): ResourceInformation = {
72-
implicit val formats = DefaultFormats
72+
implicit val formats: Formats = DefaultFormats
7373
try {
7474
parse(json).extract[ResourceInformationJson].toResourceInformation
7575
} catch {
@@ -80,7 +80,7 @@ private[spark] object ResourceInformation {
8080
}
8181

8282
def parseJson(json: JValue): ResourceInformation = {
83-
implicit val formats = DefaultFormats
83+
implicit val formats: Formats = DefaultFormats
8484
try {
8585
json.extract[ResourceInformationJson].toResourceInformation
8686
} catch {

core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.Optional
2222

2323
import scala.util.control.NonFatal
2424

25-
import org.json4s.DefaultFormats
25+
import org.json4s.{DefaultFormats, Formats}
2626
import org.json4s.jackson.JsonMethods._
2727

2828
import org.apache.spark.{SparkConf, SparkException}
@@ -252,7 +252,7 @@ private[spark] object ResourceUtils extends Logging {
252252

253253
def parseAllocatedFromJsonFile(resourcesFile: String): Seq[ResourceAllocation] = {
254254
withResourcesJson[ResourceAllocation](resourcesFile) { json =>
255-
implicit val formats = DefaultFormats
255+
implicit val formats: Formats = DefaultFormats
256256
parse(json).extract[Seq[ResourceAllocation]]
257257
}
258258
}

core/src/main/scala/org/apache/spark/status/AppStatusSource.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] {
3131

3232
private[spark] class AppStatusSource extends Source {
3333

34-
override implicit val metricRegistry = new MetricRegistry()
34+
override implicit val metricRegistry: MetricRegistry = new MetricRegistry()
3535

3636
override val sourceName = "appStatus"
3737

core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -21,15 +21,15 @@ import java.io.IOException
2121
import java.util.{HashMap => JHashMap}
2222
import java.util.concurrent.TimeUnit
2323

24-
import scala.collection.JavaConverters._
2524
import scala.collection.mutable
26-
import scala.concurrent.{ExecutionContext, Future, TimeoutException}
25+
import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future, TimeoutException}
26+
import scala.jdk.CollectionConverters._
2727
import scala.util.Random
2828
import scala.util.control.NonFatal
2929

3030
import com.google.common.cache.CacheBuilder
3131

32-
import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext}
32+
import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SparkEnv}
3333
import org.apache.spark.annotation.DeveloperApi
3434
import org.apache.spark.internal.{config, Logging}
3535
import org.apache.spark.internal.config.RDD_CACHE_VISIBILITY_TRACKING_ENABLED
@@ -40,6 +40,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedClusterMessages, CoarseG
4040
import org.apache.spark.shuffle.ShuffleManager
4141
import org.apache.spark.storage.BlockManagerMessages._
4242
import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils}
43+
import org.apache.spark.util.ArrayImplicits._
4344

4445
/**
4546
* BlockManagerMasterEndpoint is an [[IsolatedThreadSafeRpcEndpoint]] on the master node to
@@ -54,10 +55,15 @@ class BlockManagerMasterEndpoint(
5455
externalBlockStoreClient: Option[ExternalBlockStoreClient],
5556
blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo],
5657
mapOutputTracker: MapOutputTrackerMaster,
57-
shuffleManager: ShuffleManager,
58+
private val _shuffleManager: ShuffleManager,
5859
isDriver: Boolean)
5960
extends IsolatedThreadSafeRpcEndpoint with Logging {
6061

62+
// We initialize the ShuffleManager later in SparkContext and Executor, to allow
63+
// user jars to define custom ShuffleManagers, as such `_shuffleManager` will be null here
64+
// (except for tests) and we ask for the instance from the SparkEnv.
65+
private lazy val shuffleManager = Option(_shuffleManager).getOrElse(SparkEnv.get.shuffleManager)
66+
6167
// Mapping from executor id to the block manager's local disk directories.
6268
private val executorIdToLocalDirs =
6369
CacheBuilder
@@ -94,7 +100,8 @@ class BlockManagerMasterEndpoint(
94100

95101
private val askThreadPool =
96102
ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool", 100)
97-
private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool)
103+
private implicit val askExecutionContext: ExecutionContextExecutorService =
104+
ExecutionContext.fromExecutorService(askThreadPool)
98105

99106
private val topologyMapper = {
100107
val topologyMapperClassName = conf.get(
@@ -877,7 +884,7 @@ class BlockManagerMasterEndpoint(
877884

878885
private def getLocationsMultipleBlockIds(
879886
blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = {
880-
blockIds.map(blockId => getLocations(blockId))
887+
blockIds.map(blockId => getLocations(blockId)).toImmutableArraySeq
881888
}
882889

883890
/** Get the list of the peers of the given block manager */

core/src/main/scala/org/apache/spark/storage/BlockManagerStorageEndpoint.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.storage
1919

20-
import scala.concurrent.{ExecutionContext, Future}
20+
import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future}
2121

2222
import org.apache.spark.{MapOutputTracker, SparkEnv}
2323
import org.apache.spark.internal.Logging
@@ -38,7 +38,8 @@ class BlockManagerStorageEndpoint(
3838

3939
private val asyncThreadPool =
4040
ThreadUtils.newDaemonCachedThreadPool("block-manager-storage-async-thread-pool", 100)
41-
private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool)
41+
private implicit val asyncExecutionContext: ExecutionContextExecutorService =
42+
ExecutionContext.fromExecutorService(asyncThreadPool)
4243

4344
// Operations that involve removing blocks may be slow and should be done asynchronously
4445
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {

core/src/main/scala/org/apache/spark/util/ThreadUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -379,7 +379,7 @@ private[spark] object ThreadUtils {
379379
def parmap[I, O](in: Seq[I], prefix: String, maxThreads: Int)(f: I => O): Seq[O] = {
380380
val pool = newForkJoinPool(prefix, maxThreads)
381381
try {
382-
implicit val ec = ExecutionContext.fromExecutor(pool)
382+
implicit val ec: ExecutionContextExecutor = ExecutionContext.fromExecutor(pool)
383383

384384
val futures = in.map(x => Future(f(x)))
385385
val futureSeq = Future.sequence(futures)

core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ import org.apache.spark.storage._
4242
abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[SortShuffleManager])
4343
extends SparkFunSuite with BeforeAndAfter with LocalSparkContext
4444
{
45-
implicit val defaultTimeout = timeout(10.seconds)
45+
implicit val defaultTimeout: PatienceConfiguration.Timeout = timeout(10.seconds)
4646
val conf = new SparkConf()
4747
.setMaster("local[2]")
4848
.setAppName("ContextCleanerSuite")

core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import java.util.function.Supplier
2323

2424
import scala.concurrent.duration._
2525

26-
import org.json4s.{DefaultFormats, Extraction}
26+
import org.json4s.{DefaultFormats, Extraction, Formats}
2727
import org.mockito.{Mock, MockitoAnnotations}
2828
import org.mockito.Answers.RETURNS_SMART_NULLS
2929
import org.mockito.ArgumentMatchers.any
@@ -60,7 +60,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter {
6060
}
6161
def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts)
6262

63-
implicit val formats = DefaultFormats
63+
implicit val formats: Formats = DefaultFormats
6464

6565
private var _worker: Worker = _
6666

core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger
2626
import scala.collection.concurrent.TrieMap
2727
import scala.concurrent.duration._
2828

29-
import org.json4s.{DefaultFormats, Extraction}
29+
import org.json4s.{DefaultFormats, Extraction, Formats}
3030
import org.json4s.JsonAST.{JArray, JObject}
3131
import org.json4s.JsonDSL._
3232
import org.mockito.ArgumentMatchers.any
@@ -50,7 +50,7 @@ import org.apache.spark.util.{SerializableBuffer, ThreadUtils, Utils}
5050
class CoarseGrainedExecutorBackendSuite extends SparkFunSuite
5151
with LocalSparkContext with MockitoSugar {
5252

53-
implicit val formats = DefaultFormats
53+
implicit val formats: Formats = DefaultFormats
5454

5555
test("parsing no resources") {
5656
val conf = new SparkConf

core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.memory
2020
import java.util.concurrent.atomic.AtomicLong
2121

2222
import scala.collection.mutable
23-
import scala.concurrent.{ExecutionContext, Future}
23+
import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
2424
import scala.concurrent.duration.Duration
2525

2626
import org.mockito.ArgumentMatchers.{any, anyLong}
@@ -148,7 +148,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite {
148148
// -- Tests of sharing of execution memory between tasks ----------------------------------------
149149
// Prior to Spark 1.6, these tests were part of ShuffleMemoryManagerSuite.
150150

151-
implicit val ec = ExecutionContext.global
151+
implicit val ec: ExecutionContextExecutor = ExecutionContext.global
152152

153153
test("single task requesting on-heap execution memory") {
154154
val manager = createMemoryManager(1000L)

core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.storage
1919

2020
import java.util.Properties
2121

22-
import scala.concurrent.{ExecutionContext, Future}
22+
import scala.concurrent.{ExecutionContext, ExecutionContextExecutor, Future}
2323
import scala.language.implicitConversions
2424
import scala.reflect.ClassTag
2525

@@ -31,7 +31,7 @@ import org.apache.spark.util.ThreadUtils
3131

3232
class BlockInfoManagerSuite extends SparkFunSuite {
3333

34-
private implicit val ec = ExecutionContext.global
34+
private implicit val ec: ExecutionContextExecutor = ExecutionContext.global
3535
private var blockInfoManager: BlockInfoManager = _
3636

3737
override protected def beforeEach(): Unit = {

core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@ private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler {
8383
class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers {
8484

8585
implicit var webDriver: WebDriver = _
86-
implicit val formats = DefaultFormats
86+
implicit val formats: Formats = DefaultFormats
8787

8888

8989
override def beforeAll(): Unit = {

core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,14 +22,14 @@ import java.util.concurrent.atomic.AtomicInteger
2222

2323
import scala.concurrent.duration._
2424

25-
import org.scalatest.concurrent.{ThreadSignaler, TimeLimits}
25+
import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits}
2626

2727
import org.apache.spark.SparkFunSuite
2828

2929
class KeyLockSuite extends SparkFunSuite with TimeLimits {
3030

3131
// Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x
32-
private implicit val defaultSignaler = ThreadSignaler
32+
private implicit val defaultSignaler: Signaler = ThreadSignaler
3333

3434
private val foreverMs = 60 * 1000L
3535

examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
package org.apache.spark.examples.sql
1818

1919
// $example on:programmatic_schema$
20-
import org.apache.spark.sql.Row
20+
import org.apache.spark.sql.{Encoder, Row}
2121
// $example off:programmatic_schema$
2222
// $example on:init_session$
2323
import org.apache.spark.sql.SparkSession
@@ -220,7 +220,8 @@ object SparkSQLExample {
220220
// +------------+
221221

222222
// No pre-defined encoders for Dataset[Map[K,V]], define explicitly
223-
implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
223+
implicit val mapEncoder: Encoder[Map[String, Any]] =
224+
org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
224225
// Primitive types and case classes can be also defined as
225226
// implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder()
226227

mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.Locale
2222
import breeze.linalg.normalize
2323
import breeze.numerics.exp
2424
import org.apache.hadoop.fs.Path
25-
import org.json4s.DefaultFormats
25+
import org.json4s.{DefaultFormats, Formats}
2626
import org.json4s.JsonAST.JObject
2727
import org.json4s.jackson.JsonMethods._
2828

@@ -384,7 +384,7 @@ private object LDAParams {
384384
def getAndSetParams(model: LDAParams, metadata: Metadata): Unit = {
385385
VersionUtils.majorMinorVersion(metadata.sparkVersion) match {
386386
case (1, 6) =>
387-
implicit val format = DefaultFormats
387+
implicit val format: Formats = DefaultFormats
388388
metadata.params match {
389389
case JObject(pairs) =>
390390
pairs.foreach { case (paramName, jsonValue) =>

mllib/src/main/scala/org/apache/spark/ml/linalg/JsonMatrixConverter.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
*/
1717
package org.apache.spark.ml.linalg
1818

19-
import org.json4s.DefaultFormats
19+
import org.json4s.{DefaultFormats, Formats}
2020
import org.json4s.JsonDSL._
2121
import org.json4s.jackson.JsonMethods.{compact, parse => parseJson, render}
2222

@@ -29,7 +29,7 @@ private[ml] object JsonMatrixConverter {
2929
* Parses the JSON representation of a Matrix into a [[Matrix]].
3030
*/
3131
def fromJson(json: String): Matrix = {
32-
implicit val formats = DefaultFormats
32+
implicit val formats: Formats = DefaultFormats
3333
val jValue = parseJson(json)
3434
(jValue \ "type").extract[Int] match {
3535
case 0 => // sparse

0 commit comments

Comments
 (0)