Skip to content

Commit 5ae58a3

Browse files
committed
Fix merge
1 parent 1156291 commit 5ae58a3

File tree

3 files changed

+47
-16
lines changed

3 files changed

+47
-16
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.nio.charset.StandardCharsets
2121
import java.sql.{Date, Timestamp}
2222

2323
import org.apache.spark.sql.Row
24-
import org.apache.spark.sql.catalyst.util.DateTimeUtils
24+
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
2525
import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand}
2626
import org.apache.spark.sql.internal.SQLConf
2727
import org.apache.spark.sql.types._
@@ -56,6 +56,10 @@ object HiveResult {
5656
result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t"))
5757
}
5858

59+
private lazy val dateFormatter = DateFormatter()
60+
private lazy val timestampFormatter = TimestampFormatter(
61+
DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone))
62+
5963
/** Formats a datum (based on the given data type) and returns the string representation. */
6064
private def toHiveString(a: (Any, DataType)): String = {
6165
val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType,
@@ -103,10 +107,9 @@ object HiveResult {
103107
toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType))
104108
}.toSeq.sorted.mkString("{", ",", "}")
105109
case (null, _) => "NULL"
106-
case (d: Date, DateType) =>
107-
DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d))
110+
case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d))
108111
case (t: Timestamp, TimestampType) =>
109-
DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), timeZone)
112+
DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t))
110113
case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8)
111114
case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal)
112115
case (interval, CalendarIntervalType) => interval.toString
Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,40 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution
19+
20+
import java.sql.{Date, Timestamp}
21+
22+
import org.apache.spark.sql.test.SharedSQLContext
23+
24+
class HiveResultSuite extends SharedSQLContext {
25+
import testImplicits._
26+
27+
test("date formatting in hive result") {
28+
val date = "2018-12-28"
29+
val executedPlan = Seq(Date.valueOf(date)).toDS().queryExecution.executedPlan
30+
val result = HiveResult.hiveResultString(executedPlan)
31+
assert(result.head == date)
32+
}
33+
34+
test("timestamp formatting in hive result") {
35+
val timestamp = "2018-12-28 01:02:03"
36+
val executedPlan = Seq(Timestamp.valueOf(timestamp)).toDS().queryExecution.executedPlan
37+
val result = HiveResult.hiveResultString(executedPlan)
38+
assert(result.head == timestamp)
39+
}
40+
}

sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -121,18 +121,6 @@ class QueryExecutionSuite extends SharedSQLContext {
121121
}
122122
}
123123

124-
test("date formatting in hive result") {
125-
val date = "2018-12-28"
126-
val result = Seq(Date.valueOf(date)).toDS().queryExecution.hiveResultString()
127-
assert(result.head == date)
128-
}
129-
130-
test("timestamp formatting in hive result") {
131-
val timestamp = "2018-12-28 01:02:03"
132-
val result = Seq(Timestamp.valueOf(timestamp)).toDS().queryExecution.hiveResultString()
133-
assert(result.head == timestamp)
134-
}
135-
136124
test("toString() exception/error handling") {
137125
spark.experimental.extraStrategies = Seq(
138126
new SparkStrategy {

0 commit comments

Comments
 (0)