asfgit closed pull request #23466: [SPARK-26547][SQL] Remove duplicate
toHiveString from HiveUtils
URL: https://github.com/apache/spark/pull/23466
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
index 22d3ca958a210..c90b254a6d121 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
@@ -56,61 +56,70 @@ object HiveResult {
result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t"))
}
- /** Formats a datum (based on the given data type) and returns the string
representation. */
- private def toHiveString(a: (Any, DataType)): String = {
- val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType,
FloatType,
- BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType)
- val timeZone = DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone)
-
- def formatDecimal(d: java.math.BigDecimal): String = {
- if (d.compareTo(java.math.BigDecimal.ZERO) == 0) {
- java.math.BigDecimal.ZERO.toPlainString
- } else {
- d.stripTrailingZeros().toPlainString
- }
+ private def formatDecimal(d: java.math.BigDecimal): String = {
+ if (d.compareTo(java.math.BigDecimal.ZERO) == 0) {
+ java.math.BigDecimal.ZERO.toPlainString
+ } else {
+ d.stripTrailingZeros().toPlainString // Hive strips trailing zeros
}
+ }
- /** Hive outputs fields of structs slightly differently than top level
attributes. */
- def toHiveStructString(a: (Any, DataType)): String = a match {
- case (struct: Row, StructType(fields)) =>
- struct.toSeq.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_, _], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" +
toHiveStructString((value, vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "null"
- case (s: String, StringType) => "\"" + s + "\""
- case (decimal, DecimalType()) => decimal.toString
- case (interval, CalendarIntervalType) => interval.toString
- case (other, tpe) if primitiveTypes contains tpe => other.toString
- }
+ private val primitiveTypes = Seq(
+ StringType,
+ IntegerType,
+ LongType,
+ DoubleType,
+ FloatType,
+ BooleanType,
+ ByteType,
+ ShortType,
+ DateType,
+ TimestampType,
+ BinaryType)
- a match {
- case (struct: Row, StructType(fields)) =>
- struct.toSeq.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_, _], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" +
toHiveStructString((value, vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "NULL"
- case (d: Date, DateType) =>
- DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d))
- case (t: Timestamp, TimestampType) =>
- DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t),
timeZone)
- case (bin: Array[Byte], BinaryType) => new String(bin,
StandardCharsets.UTF_8)
- case (decimal: java.math.BigDecimal, DecimalType()) =>
formatDecimal(decimal)
- case (interval, CalendarIntervalType) => interval.toString
- case (other, tpe) if primitiveTypes.contains(tpe) => other.toString
- }
+ /** Hive outputs fields of structs slightly differently than top level
attributes. */
+ private def toHiveStructString(a: (Any, DataType)): String = a match {
+ case (struct: Row, StructType(fields)) =>
+ struct.toSeq.zip(fields).map {
+ case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
+ }.mkString("{", ",", "}")
+ case (seq: Seq[_], ArrayType(typ, _)) =>
+ seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
+ case (map: Map[_, _], MapType(kType, vType, _)) =>
+ map.map {
+ case (key, value) =>
+ toHiveStructString((key, kType)) + ":" + toHiveStructString((value,
vType))
+ }.toSeq.sorted.mkString("{", ",", "}")
+ case (null, _) => "null"
+ case (s: String, StringType) => "\"" + s + "\""
+ case (decimal, DecimalType()) => decimal.toString
+ case (interval, CalendarIntervalType) => interval.toString
+ case (other, tpe) if primitiveTypes contains tpe => other.toString
+ }
+
+ /** Formats a datum (based on the given data type) and returns the string
representation. */
+ def toHiveString(a: (Any, DataType)): String = a match {
+ case (struct: Row, StructType(fields)) =>
+ struct.toSeq.zip(fields).map {
+ case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
+ }.mkString("{", ",", "}")
+ case (seq: Seq[_], ArrayType(typ, _)) =>
+ seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
+ case (map: Map[_, _], MapType(kType, vType, _)) =>
+ map.map {
+ case (key, value) =>
+ toHiveStructString((key, kType)) + ":" + toHiveStructString((value,
vType))
+ }.toSeq.sorted.mkString("{", ",", "}")
+ case (null, _) => "NULL"
+ case (d: Date, DateType) =>
+ DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d))
+ case (t: Timestamp, TimestampType) =>
+ val timeZone =
DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone)
+ DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t),
timeZone)
+ case (bin: Array[Byte], BinaryType) => new String(bin,
StandardCharsets.UTF_8)
+ case (decimal: java.math.BigDecimal, DecimalType()) =>
formatDecimal(decimal)
+ case (interval, CalendarIntervalType) => interval.toString
+ case (other, _ : UserDefinedType[_]) => other.toString
+ case (other, tpe) if primitiveTypes.contains(tpe) => other.toString
}
}
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
new file mode 100644
index 0000000000000..4205b3f79a972
--- /dev/null
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT}
+
+class HiveResultSuite extends SparkFunSuite {
+
+ test("toHiveString correctly handles UDTs") {
+ val point = new ExamplePoint(50.0, 50.0)
+ val tpe = new ExamplePointUDT()
+ assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)")
+ }
+}
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 3cfc81b8a9579..e68c6011c1393 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
@@ -34,8 +34,8 @@ import org.apache.hive.service.cli.session.HiveSession
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext}
+import org.apache.spark.sql.execution.HiveResult
import org.apache.spark.sql.execution.command.SetCommand
-import org.apache.spark.sql.hive.HiveUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.{Utils => SparkUtils}
@@ -103,7 +103,7 @@ private[hive] class SparkExecuteStatementOperation(
case BinaryType =>
to += from.getAs[Array[Byte]](ordinal)
case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] =>
- val hiveString = HiveUtils.toHiveString((from.get(ordinal),
dataTypes(ordinal)))
+ val hiveString = HiveResult.toHiveString((from.get(ordinal),
dataTypes(ordinal)))
to += hiveString
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
index b60d4c71f5941..597eef129f63e 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
@@ -444,52 +444,6 @@ private[spark] object HiveUtils extends Logging {
propMap.toMap
}
- protected val primitiveTypes =
- Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType,
ByteType,
- ShortType, DateType, TimestampType, BinaryType)
-
- protected[sql] def toHiveString(a: (Any, DataType)): String = a match {
- case (struct: Row, StructType(fields)) =>
- struct.toSeq.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_, _], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" + toHiveStructString((value,
vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "NULL"
- case (d: Int, DateType) => new DateWritable(d).toString
- case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString
- case (bin: Array[Byte], BinaryType) => new String(bin,
StandardCharsets.UTF_8)
- case (decimal: java.math.BigDecimal, DecimalType()) =>
- // Hive strips trailing zeros so use its toString
- HiveDecimal.create(decimal).toString
- case (other, _ : UserDefinedType[_]) => other.toString
- case (other, tpe) if primitiveTypes contains tpe => other.toString
- }
-
- /** Hive outputs fields of structs slightly differently than top level
attributes. */
- protected def toHiveStructString(a: (Any, DataType)): String = a match {
- case (struct: Row, StructType(fields)) =>
- struct.toSeq.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString((v,
t.dataType))}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_, _], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" + toHiveStructString((value,
vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "null"
- case (s: String, StringType) => "\"" + s + "\""
- case (decimal, DecimalType()) => decimal.toString
- case (other, tpe) if primitiveTypes contains tpe => other.toString
- }
-
/**
* Infers the schema for Hive serde tables and returns the CatalogTable with
the inferred schema.
* When the tables are data source tables or the schema already exists,
returns the original
diff --git
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
index f2b75e4b23f02..303dd70760a1b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala
@@ -17,16 +17,15 @@
package org.apache.spark.sql.hive
-import java.net.URL
-
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.spark.SparkConf
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.execution.HiveResult
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SQLTestUtils}
-import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader}
+import org.apache.spark.util.ChildFirstURLClassLoader
class HiveUtilsSuite extends QueryTest with SQLTestUtils with
TestHiveSingleton {
@@ -62,10 +61,4 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils
with TestHiveSingleton
Thread.currentThread().setContextClassLoader(contextClassLoader)
}
}
-
- test("toHiveString correctly handles UDTs") {
- val point = new ExamplePoint(50.0, 50.0)
- val tpe = new ExamplePointUDT()
- assert(HiveUtils.toHiveString((point, tpe)) === "(50.0, 50.0)")
- }
}
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]