aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src
diff options
context:
space:
mode:
Diffstat (limited to 'sql/core/src')
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala9
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala10
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala1
8 files changed, 24 insertions, 15 deletions
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java
index 7d768b165f..7234726633 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java
@@ -846,8 +846,9 @@ public class UnsafeRowParquetRecordReader extends SpecificParquetRecordReaderBas
" as the dictionary was missing for encoding " + dataEncoding);
}
if (vectorizedDecode()) {
- if (dataEncoding != Encoding.PLAIN_DICTIONARY &&
- dataEncoding != Encoding.RLE_DICTIONARY) {
+ @SuppressWarnings("deprecation")
+ Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression
+ if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) {
throw new NotImplementedException("Unsupported encoding: " + dataEncoding);
}
this.dataColumn = new VectorizedRleValuesReader();
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
index 8c46516594..da28ec4f53 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution.python
import java.io.OutputStream
+import java.nio.charset.StandardCharsets
import scala.collection.JavaConverters._
@@ -136,7 +137,7 @@ object EvaluatePython {
case (c, StringType) => UTF8String.fromString(c.toString)
- case (c: String, BinaryType) => c.getBytes("utf-8")
+ case (c: String, BinaryType) => c.getBytes(StandardCharsets.UTF_8)
case (c, BinaryType) if c.getClass.isArray && c.getClass.getComponentType.getName == "byte" => c
case (c: java.util.List[_], ArrayType(elementType, _)) =>
@@ -185,7 +186,8 @@ object EvaluatePython {
def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
out.write(Opcodes.GLOBAL)
- out.write((module + "\n" + "_parse_datatype_json_string" + "\n").getBytes("utf-8"))
+ out.write(
+ (module + "\n" + "_parse_datatype_json_string" + "\n").getBytes(StandardCharsets.UTF_8))
val schema = obj.asInstanceOf[StructType]
pickler.save(schema.json)
out.write(Opcodes.TUPLE1)
@@ -209,7 +211,8 @@ object EvaluatePython {
def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
if (obj == this) {
out.write(Opcodes.GLOBAL)
- out.write((module + "\n" + "_create_row_inbound_converter" + "\n").getBytes("utf-8"))
+ out.write(
+ (module + "\n" + "_create_row_inbound_converter" + "\n").getBytes(StandardCharsets.UTF_8))
} else {
// it will be memorized by Pickler to save some bytes
pickler.save(this)
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 9ca8c4d2ed..9d7570fe7a 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
@@ -18,6 +18,7 @@
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.sql.Row
@@ -67,7 +68,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging {
checkActualSize(LONG, Long.MaxValue, 8)
checkActualSize(FLOAT, Float.MaxValue, 4)
checkActualSize(DOUBLE, Double.MaxValue, 8)
- checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length)
+ checkActualSize(STRING, "hello", 4 + "hello".getBytes(StandardCharsets.UTF_8).length)
checkActualSize(BINARY, Array.fill[Byte](4)(0.toByte), 4 + 4)
checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8)
checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index 6e21d5a061..0940878e38 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.columnar
+import java.nio.charset.StandardCharsets
import java.sql.{Date, Timestamp}
import org.apache.spark.sql.{QueryTest, Row}
@@ -160,7 +161,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
sparkContext.parallelize((1 to 10000), 10).map { i =>
Row(
s"str${i}: test cache.",
- s"binary${i}: test cache.".getBytes("UTF-8"),
+ s"binary${i}: test cache.".getBytes(StandardCharsets.UTF_8),
null,
i % 2 == 0,
i.toByte,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index 4671b2dca9..4a8c128fa9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution.datasources.json
import java.io.{File, StringWriter}
+import java.nio.charset.StandardCharsets
import java.sql.{Date, Timestamp}
import scala.collection.JavaConverters._
@@ -27,7 +28,6 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.io.compress.GzipCodec
-import org.scalactic.Tolerance._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
@@ -1292,7 +1292,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
val constantValues =
Seq(
- "a string in binary".getBytes("UTF-8"),
+ "a string in binary".getBytes(StandardCharsets.UTF_8),
null,
true,
1.toByte,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
index 36b929ee1f..f98ea8c5ae 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
@@ -17,8 +17,8 @@
package org.apache.spark.sql.execution.datasources.parquet
-import java.io.File
import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
import java.util.{List => JList, Map => JMap}
import scala.collection.JavaConverters._
@@ -59,7 +59,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared
.setLongColumn(i.toLong * 10)
.setFloatColumn(i.toFloat + 0.1f)
.setDoubleColumn(i.toDouble + 0.2d)
- .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8")))
+ .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes(StandardCharsets.UTF_8)))
.setStringColumn(s"val_$i")
.build())
}
@@ -74,7 +74,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared
i.toLong * 10,
i.toFloat + 0.1f,
i.toDouble + 0.2d,
- s"val_$i".getBytes("UTF-8"),
+ s"val_$i".getBytes(StandardCharsets.UTF_8),
s"val_$i")
})
}
@@ -103,7 +103,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared
.setMaybeLongColumn(i.toLong * 10)
.setMaybeFloatColumn(i.toFloat + 0.1f)
.setMaybeDoubleColumn(i.toDouble + 0.2d)
- .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8")))
+ .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes(StandardCharsets.UTF_8)))
.setMaybeStringColumn(s"val_$i")
.build()
}
@@ -124,7 +124,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared
i.toLong * 10,
i.toFloat + 0.1f,
i.toDouble + 0.2d,
- s"val_$i".getBytes("UTF-8"),
+ s"val_$i".getBytes(StandardCharsets.UTF_8),
s"val_$i")
}
})
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index a64df435d8..b394ffb366 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.execution.datasources.parquet
+import java.nio.charset.StandardCharsets
+
import org.apache.parquet.filter2.predicate.{FilterPredicate, Operators}
import org.apache.parquet.filter2.predicate.FilterApi._
import org.apache.parquet.filter2.predicate.Operators.{Column => _, _}
@@ -260,7 +262,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex
// See https://issues.apache.org/jira/browse/SPARK-11153
ignore("filter pushdown - binary") {
implicit class IntToBinary(int: Int) {
- def b: Array[Byte] = int.toString.getBytes("UTF-8")
+ def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8)
}
withParquetDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
index 547d06236b..1c8b2ea808 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
@@ -81,6 +81,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext {
val buildSide = joinType match {
case LeftOuter => BuildRight
case RightOuter => BuildLeft
+ case _ => fail(s"Unsupported join type $joinType")
}
extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) =>
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {