Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
c78a347
WIP
hvanhovell Jul 25, 2023
0eb42c1
Fix conflicts
hvanhovell Jul 26, 2023
f648bef
Finish move of classes
hvanhovell Jul 26, 2023
be5b320
Fix legacy policy
hvanhovell Jul 26, 2023
c9fc20c
Remove ToJsonUtil
hvanhovell Jul 26, 2023
0b1b803
Add UDTUtilsImpl
hvanhovell Jul 26, 2023
783a1af
Fix stuff
hvanhovell Jul 26, 2023
4fa9124
Merge with master
hvanhovell Jul 26, 2023
4fc0e6d
Simplification & Fixes
hvanhovell Jul 26, 2023
65ea542
Disconnect Arrow encoders from Catalyst
hvanhovell Jul 26, 2023
b70413b
Disconnect Arrow encoders from Catalyst
hvanhovell Jul 26, 2023
67d1592
Fix proto functions
hvanhovell Jul 26, 2023
340304d
Fix proto functions
hvanhovell Jul 26, 2023
8ca1d2b
fix proto for real
hvanhovell Jul 26, 2023
4845456
Make tests compile!
hvanhovell Jul 26, 2023
42d8f0f
Capstone
hvanhovell Jul 26, 2023
69b8868
Fix compilation
hvanhovell Jul 26, 2023
1927189
Style
hvanhovell Jul 26, 2023
a4f3050
Fix formatting
hvanhovell Jul 27, 2023
ad78a8d
Merge branch 'SPARK-44538' into SPARK-41400-v1
hvanhovell Jul 27, 2023
9f10a8c
Merge remote-tracking branch 'apache/master' into SPARK-41400-v1
hvanhovell Jul 27, 2023
cc981b4
Integrate AnalyisException
hvanhovell Jul 27, 2023
bdd4346
Merge with master
hvanhovell Jul 27, 2023
2b8879d
Put back DateTimeUtils and enrich SparkDateTimeUtils.
hvanhovell Jul 27, 2023
d6a57a0
Merge remote-tracking branch 'apache/master' into SPARK-44538
hvanhovell Jul 27, 2023
e2b0dc4
Merge branch 'SPARK-44538' into SPARK-41400-v1
hvanhovell Jul 27, 2023
fcda03c
Put back string to/from bytebuffer
hvanhovell Jul 27, 2023
912f892
Undo change to RebaseDateTimeSuite
hvanhovell Jul 27, 2023
eeaa409
Undo change
hvanhovell Jul 27, 2023
30f3ce4
style...
hvanhovell Jul 28, 2023
6b53488
Merge branch 'SPARK-44538' into SPARK-41400-v1
hvanhovell Jul 28, 2023
470fed4
Merge remote-tracking branch 'apache/master' into SPARK-41400-v1
hvanhovell Jul 28, 2023
c81d847
Fix docs
hvanhovell Jul 28, 2023
064bc40
Merge remote-tracking branch 'apache/master' into SPARK-41400-v1
hvanhovell Jul 28, 2023
ace8fed
Weird Bug Fix
hvanhovell Jul 28, 2023
5c5aa60
Better fix
hvanhovell Jul 28, 2023
b7bc5c4
Fix MiMa
hvanhovell Jul 28, 2023
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
WIP
  • Loading branch information
hvanhovell committed Jul 25, 2023
commit c78a347b9b0c0e16058bd9759e6120c227037cbb
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.DataSourceUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.sql.internal.LegacyBehaviorPolicy
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String

Expand Down
87 changes: 87 additions & 0 deletions sql/api/src/main/scala/org/apache/spark/sql/Row.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,24 @@

package org.apache.spark.sql

import java.sql.{Date, Timestamp}
import java.time.{Instant, LocalDate}
import java.util.Base64

import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.hashing.MurmurHash3

import org.json4s.{JArray, JBool, JDecimal, JDouble, JField, JLong, JNull, JObject, JString}
import org.json4s.JsonAST.JValue

import org.apache.spark.annotation.Stable
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter, UDTUtils}
import org.apache.spark.sql.errors.DataTypeErrors
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval

/**
* @since 1.3.0
Expand Down Expand Up @@ -513,4 +524,80 @@ trait Row extends Serializable {
private def getAnyValAs[T <: AnyVal](i: Int): T =
if (isNullAt(i)) throw DataTypeErrors.valueIsNullError(i)
else getAs[T](i)

/**
* JSON representation of the row.
*
* Note that this only supports the data types that are also supported by
* [[org.apache.spark.sql.catalyst.encoders.RowEncoder]].
*
* @return the JSON representation of the row.
*/
private[sql] def jsonValue(row: Row): JValue = {
require(row.schema != null, "JSON serialization requires a non-null schema.")

lazy val zoneId = DateTimeUtils.getZoneId(SqlApiConf.get.sessionLocalTimeZone)
lazy val dateFormatter = DateFormatter()
lazy val timestampFormatter = TimestampFormatter(zoneId)

// Convert an iterator of values to a json array
def iteratorToJsonArray(iterator: Iterator[_], elementType: DataType): JArray = {
JArray(iterator.map(toJson(_, elementType)).toList)
}

// Convert a value to json.
def toJson(value: Any, dataType: DataType): JValue = (value, dataType) match {
case (null, _) => JNull
case (b: Boolean, _) => JBool(b)
case (b: Byte, _) => JLong(b)
case (s: Short, _) => JLong(s)
case (i: Int, _) => JLong(i)
case (l: Long, _) => JLong(l)
case (f: Float, _) => JDouble(f)
case (d: Double, _) => JDouble(d)
case (d: BigDecimal, _) => JDecimal(d)
case (d: java.math.BigDecimal, _) => JDecimal(d)
case (d: Decimal, _) => JDecimal(d.toBigDecimal)
case (s: String, _) => JString(s)
case (b: Array[Byte], BinaryType) =>
JString(Base64.getEncoder.encodeToString(b))
case (d: LocalDate, _) => JString(dateFormatter.format(d))
case (d: Date, _) => JString(dateFormatter.format(d))
case (i: Instant, _) => JString(timestampFormatter.format(i))
case (t: Timestamp, _) => JString(timestampFormatter.format(t))
case (i: CalendarInterval, _) => JString(i.toString)
case (a: Array[_], ArrayType(elementType, _)) =>
iteratorToJsonArray(a.iterator, elementType)
case (a: mutable.ArraySeq[_], ArrayType(elementType, _)) =>
iteratorToJsonArray(a.iterator, elementType)
case (s: Seq[_], ArrayType(elementType, _)) =>
iteratorToJsonArray(s.iterator, elementType)
case (m: Map[String @unchecked, _], MapType(StringType, valueType, _)) =>
new JObject(m.toList.sortBy(_._1).map {
case (k, v) => k -> toJson(v, valueType)
})
case (m: Map[_, _], MapType(keyType, valueType, _)) =>
new JArray(m.iterator.map {
case (k, v) =>
new JObject("key" -> toJson(k, keyType) :: "value" -> toJson(v, valueType) :: Nil)
}.toList)
case (r: Row, _) => jsonValue(r)
case (v: Any, udt: UserDefinedType[Any @unchecked]) =>
UDTUtils.toJson(v, udt)
case _ =>
throw new IllegalArgumentException(s"Failed to convert value $value " +
s"(class of ${value.getClass}}) with the type of $dataType to JSON.")
}

// Convert the row fields to json
var n = 0
val elements = new mutable.ListBuffer[JField]
val len = row.length
while (n < len) {
val field = row.schema(n)
elements += (field.name -> toJson(row.apply(n), field.dataType))
n += 1
}
new JObject(elements.toList)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,10 @@ import scala.collection.JavaConverters._
import org.antlr.v4.runtime.Token
import org.antlr.v4.runtime.tree.ParseTree

import org.apache.spark.sql.SqlApiConf
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.util.SparkParserUtils.{string, withOrigin}
import org.apache.spark.sql.errors.QueryParsingErrors
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, ByteType, CalendarIntervalType, CharType, DataType, DateType, DayTimeIntervalType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, MetadataBuilder, NullType, ShortType, StringType, StructField, StructType, TimestampNTZType, TimestampType, VarcharType, YearMonthIntervalType}

class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,10 @@ import org.antlr.v4.runtime.tree.TerminalNodeImpl

import org.apache.spark.{QueryContext, SparkException, SparkThrowable, SparkThrowableHelper}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SqlApiConf
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, WithOrigin}
import org.apache.spark.sql.catalyst.util.SparkParserUtils
import org.apache.spark.sql.errors.QueryParsingErrors
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types.{DataType, StructType}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@ import java.util.{Date, Locale}
import org.apache.commons.lang3.time.FastDateFormat

import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.unsafe.types.UTF8String

sealed trait DateFormatter extends Serializable {
Expand Down Expand Up @@ -176,7 +176,7 @@ object DateFormatter {
locale: Locale = defaultLocale,
legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT,
isParsing: Boolean): DateFormatter = {
if (SQLConf.get.legacyTimeParserPolicy == LEGACY) {
if (SqlApiConf.get.legacyTimeParserPolicy == LEGACY) {
getLegacyFormatter(format.getOrElse(defaultPattern), locale, legacyFormat)
} else {
val df = format
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,14 @@ import java.time._
import java.time.chrono.IsoChronology
import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle}
import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries}
import java.util.{Date, Locale}

import com.google.common.cache.CacheBuilder
import java.util
import java.util.{Collections, Date, Locale}

import org.apache.spark.SPARK_DOC_ROOT
import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.SqlApiConf

trait DateTimeFormatterHelper {
private def getOrDefault(accessor: TemporalAccessor, field: ChronoField, default: Int): Int = {
Expand Down Expand Up @@ -122,16 +121,11 @@ trait DateTimeFormatterHelper {
val newPattern = convertIncompatiblePattern(pattern, isParsing)
val useVarLen = isParsing && newPattern.contains('S')
val key = (newPattern, locale, useVarLen)
var formatter = cache.getIfPresent(key)
if (formatter == null) {
formatter = buildFormatter(newPattern, locale, useVarLen)
cache.put(key, formatter)
}
formatter
cache.computeIfAbsent(key, _ => buildFormatter(newPattern, locale, useVarLen))
}

private def needConvertToSparkUpgradeException(e: Throwable): Boolean = e match {
case _: DateTimeException if SQLConf.get.legacyTimeParserPolicy == EXCEPTION => true
case _: DateTimeException if SqlApiConf.get.legacyTimeParserPolicy == EXCEPTION => true
case _ => false
}
// When legacy time parser policy set to EXCEPTION, check whether we will get different results
Expand Down Expand Up @@ -196,9 +190,14 @@ trait DateTimeFormatterHelper {
}

private object DateTimeFormatterHelper {
val cache = CacheBuilder.newBuilder()
.maximumSize(128)
.build[(String, Locale, Boolean), DateTimeFormatter]()

private val cache: util.Map[(String, Locale, Boolean), DateTimeFormatter] =
Collections.synchronizedMap(new util.LinkedHashMap {
override def removeEldestEntry(
eldest: util.Map.Entry[(String, Locale, Boolean), DateTimeFormatter]): Boolean = {
size() > 128
}
})

final val extractor = "^([^S]*)(S*)(.*)$".r

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@ import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModul

import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.util.Utils
import org.apache.spark.sql.internal.LegacyBehaviorPolicy
import org.apache.spark.util.SparkClassUtils

/**
* The collection of functions for rebasing days and microseconds from/to the hybrid calendar
Expand Down Expand Up @@ -278,7 +278,7 @@ object RebaseDateTime {
// `JsonRebaseRecord`. AnyRefMap is used here instead of Scala's immutable map because
// it is 2 times faster in DateTimeRebaseBenchmark.
private[sql] def loadRebaseRecords(fileName: String): AnyRefMap[String, RebaseInfo] = {
val file = Utils.getSparkClassLoader.getResource(fileName)
val file = SparkClassUtils.getSparkClassLoader.getResource(fileName)
val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file)
val anyRefMap = new AnyRefMap[String, RebaseInfo]((3 * jsonRebaseRecords.size) / 2)
jsonRebaseRecords.foreach { jsonRecord =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,8 @@
*/
package org.apache.spark.sql.catalyst.util

import org.apache.spark.sql.SqlApiConf
import org.apache.spark.sql.errors.DataTypeErrors
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types.{ArrayType, CharType, DataType, MapType, StringType, StructType, VarcharType}

trait SparkCharVarcharUtils {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.catalyst.util.LegacyDateFormats.{LegacyDateFormat, LENIENT_SIMPLE_DATE_FORMAT}
import org.apache.spark.sql.catalyst.util.RebaseDateTime._
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.LegacyBehaviorPolicy._
import org.apache.spark.sql.internal.SqlApiConf
import org.apache.spark.sql.types.{Decimal, TimestampNTZType}
import org.apache.spark.unsafe.types.UTF8String

Expand Down Expand Up @@ -507,7 +507,7 @@ object TimestampFormatter {
legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT,
isParsing: Boolean,
forTimestampNTZ: Boolean = false): TimestampFormatter = {
val formatter = if (SQLConf.get.legacyTimeParserPolicy == LEGACY && !forTimestampNTZ) {
val formatter = if (SqlApiConf.get.legacyTimeParserPolicy == LEGACY && !forTimestampNTZ) {
getLegacyFormatter(format.getOrElse(defaultPattern), zoneId, locale, legacyFormat)
} else {
format
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.catalyst.util

import scala.util.control.NonFatal

import org.json4s.JsonAST.JValue

import org.apache.spark.sql.types.UserDefinedType
import org.apache.spark.util.SparkClassUtils

/**
* Utilities for working with [[UserDefinedType]]s from within the `sql/api` project. UDTs are not
* not fully supported in `sql/api` (and connect), they can currently only be used in conjunction
* with catalyst because they (amongst others) require access to Spark SQLs internal data
* representation.
*
* This interface and its companion object provide an escape hatch for working with UDTs from within
* the api project (e.g. Row.toJSON). The companion will try to bind to an implementation of the
* interface in catalyst, if none is found it will bind to [[DefaultUDTUtils]].
*/
trait UDTUtils {
/**
* Convert a UDT to a JSON.
*/
def toJson(value: Any, udt: UserDefinedType[_]): JValue
}

object UDTUtils extends UDTUtils {
private val delegate = try {
val cls = SparkClassUtils.classForName("org.apache.spark.sql.catalyst.util.UDTUtilsImpl")
cls.getConstructor().newInstance().asInstanceOf[UDTUtils]
} catch {
case NonFatal(_) =>
DefaultUDTUtils
}

override def toJson(value: Any, udt: UserDefinedType[_]): JValue = delegate.toJson(value, udt)
}

object DefaultUDTUtils extends UDTUtils {
override def toJson(value: Any, udt: UserDefinedType[_]): JValue = {
throw new UnsupportedOperationException()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
package org.apache.spark.sql.errors

import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkNumberFormatException, SparkRuntimeException, SparkUnsupportedOperationException}
import org.apache.spark.{SparkArithmeticException, SparkDateTimeException, SparkException, SparkIllegalArgumentException, SparkNumberFormatException, SparkRuntimeException, SparkUnsupportedOperationException}
import org.apache.spark.sql.catalyst.trees.{Origin, SQLQueryContext}
import org.apache.spark.sql.catalyst.util.QuotingUtils
import org.apache.spark.sql.types.{DataType, Decimal, StringType}
Expand Down Expand Up @@ -295,4 +295,20 @@ private[sql] object DataTypeErrors extends DataTypeErrorsBase {
messageParameters = Map("operation" -> operation),
cause = null)
}

def invalidInputInCastToDatetimeError(
value: Any,
from: DataType,
to: DataType,
context: SQLQueryContext): Throwable = {
new SparkDateTimeException(
errorClass = "CAST_INVALID_INPUT",
messageParameters = Map(
"expression" -> toSQLValue(value, from),
"sourceType" -> toSQLType(from),
"targetType" -> toSQLType(to),
"ansiConfig" -> toSQLConf(Sql.ANSI_ENABLED.key)),
context = getQueryContext(context),
summary = getSummary(context))
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/*
* 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.internal

object LegacyBehaviorPolicy extends Enumeration {
val EXCEPTION, LEGACY, CORRECTED = Value
}
Loading