diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 0cd24b456fef7..a12354e6f1bfc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1589,7 +1589,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def select[U1](c1: TypedColumn[T, U1]): Dataset[U1] = { - implicit val encoder: ExpressionEncoder[T] = c1.encoder + implicit val encoder: ExpressionEncoder[U1] = c1.encoder val project = Project(c1.withInputType(exprEnc, logicalPlan.output).named :: Nil, logicalPlan) if (!encoder.isSerializedAsStructForTopLevel) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala index ad7c59bbd9f52..4bdec89182911 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala @@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} -import org.json4s.NoTypeHints +import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.sql.SparkSession @@ -82,7 +82,7 @@ case class CommitMetadata(nextBatchWatermarkMs: Long = 0) { } object CommitMetadata { - implicit val format = Serialization.formats(NoTypeHints) + implicit val format: Formats = Serialization.formats(NoTypeHints) def apply(json: String): CommitMetadata = Serialization.read[CommitMetadata](json) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala index a2b49d944a688..ba79c77f38677 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming import scala.util.control.Exception._ -import org.json4s.NoTypeHints +import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization /** @@ -34,7 +34,7 @@ case class FileStreamSourceOffset(logOffset: Long) extends Offset { } object FileStreamSourceOffset { - implicit val format = Serialization.formats(NoTypeHints) + implicit val format: Formats = Serialization.formats(NoTypeHints) def apply(offset: Offset): FileStreamSourceOffset = { offset match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index b7ea70124d271..9f1189cc51704 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -52,6 +52,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: private implicit val formats: Formats = Serialization.formats(NoTypeHints) /** Needed to serialize type T into JSON when using Jackson */ + @scala.annotation.nowarn private implicit val manifest = Manifest.classType[T](implicitly[ClassTag[T]].runtimeClass) // Avoid serializing generic sequences, see SPARK-17372 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index 913805d1a074d..9aff4b1dc5e76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming -import org.json4s.NoTypeHints +import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging @@ -88,7 +88,7 @@ case class OffsetSeqMetadata( } object OffsetSeqMetadata extends Logging { - private implicit val format = Serialization.formats(NoTypeHints) + private implicit val format: Formats = Serialization.formats(NoTypeHints) /** * These configs are related to streaming query execution and should not be changed across * batches of a streaming query. The values of these configs are persisted into the offset diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala index cb18988b46872..978cb3c34f606 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala @@ -25,7 +25,7 @@ import scala.util.control.NonFatal import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, Path} -import org.json4s.NoTypeHints +import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging @@ -45,7 +45,7 @@ case class StreamMetadata(id: String) { } object StreamMetadata extends Logging { - implicit val format = Serialization.formats(NoTypeHints) + implicit val format: Formats = Serialization.formats(NoTypeHints) /** Read the metadata from file if it exists */ def read(metadataFile: Path, hadoopConf: Configuration): Option[StreamMetadata] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 300a3b8137b4c..6181d216bd36f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -34,7 +34,7 @@ import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModul import org.apache.commons.io.{FilenameUtils, IOUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path, PathFilter} -import org.json4s.NoTypeHints +import org.json4s.{Formats, NoTypeHints} import org.json4s.jackson.Serialization import org.apache.spark.{SparkConf, SparkEnv} @@ -737,7 +737,7 @@ case class RocksDBCheckpointMetadata( object RocksDBCheckpointMetadata { val VERSION = 1 - implicit val format = Serialization.formats(NoTypeHints) + implicit val format: Formats = Serialization.formats(NoTypeHints) /** Used to convert between classes and JSON. */ lazy val mapper = {