Migration Guide: SQL, Datasets and DataFrame
- Upgrading from Spark SQL 3.4.1 to 3.4.2
- Upgrading from Spark SQL 3.3 to 3.4
- Upgrading from Spark SQL 3.2 to 3.3
- Upgrading from Spark SQL 3.1 to 3.2
- Upgrading from Spark SQL 3.0 to 3.1
- Upgrading from Spark SQL 3.0.1 to 3.0.2
- Upgrading from Spark SQL 3.0 to 3.0.1
- Upgrading from Spark SQL 2.4 to 3.0
- Upgrading from Spark SQL 2.4.7 to 2.4.8
- Upgrading from Spark SQL 2.4.5 to 2.4.6
- Upgrading from Spark SQL 2.4.4 to 2.4.5
- Upgrading from Spark SQL 2.4.3 to 2.4.4
- Upgrading from Spark SQL 2.4 to 2.4.1
- Upgrading from Spark SQL 2.3 to 2.4
- Upgrading from Spark SQL 2.2 to 2.3
- Upgrading from Spark SQL 2.1 to 2.2
- Upgrading from Spark SQL 2.0 to 2.1
- Upgrading from Spark SQL 1.6 to 2.0
- Upgrading from Spark SQL 1.5 to 1.6
- Upgrading from Spark SQL 1.4 to 1.5
- Upgrading from Spark SQL 1.3 to 1.4
- Upgrading from Spark SQL 1.0-1.2 to 1.3
- Compatibility with Apache Hive
Upgrading from Spark SQL 3.4.1 to 3.4.2
- Since Spark 3.4.2, the array_insertfunction is 1-based for negative indexes. It inserts new element at the end of input arrays for the index -1. To restore the previous behavior, setspark.sql.legacy.negativeIndexInArrayInserttotrue.
Upgrading from Spark SQL 3.3 to 3.4
- Since Spark 3.4, INSERT INTO commands with explicit column lists comprising fewer columns than the target table will automatically add the corresponding default values for the remaining columns (or NULL for any column lacking an explicitly-assigned default value). In Spark 3.3 or earlier, these commands would have failed returning errors reporting that the number of provided columns does not match the number of columns in the target table. Note that disabling spark.sql.defaultColumn.useNullsForMissingDefaultValueswill restore the previous behavior.
- Since Spark 3.4, Number or Number(*) from Teradata will be treated as Decimal(38,18). In Spark 3.3 or earlier, Number or Number(*) from Teradata will be treated as Decimal(38, 0), in which case the fractional part will be removed.
- Since Spark 3.4, v1 database, table, permanent view and function identifier will include ‘spark_catalog’ as the catalog name if database is defined, e.g. a table identifier will be: spark_catalog.default.t. To restore the legacy behavior, setspark.sql.legacy.v1IdentifierNoCatalogtotrue.
- Since Spark 3.4, when ANSI SQL mode(configuration spark.sql.ansi.enabled) is on, Spark SQL always returns NULL result on getting a map value with a non-existing key. In Spark 3.3 or earlier, there will be an error.
- Since Spark 3.4, the SQL CLI spark-sqldoes not print the prefixError in query:before the error message ofAnalysisException.
- Since Spark 3.4, splitfunction ignores trailing empty strings whenregexparameter is empty.
- Since Spark 3.4, the to_binaryfunction throws error for a malformedstrinput. Usetry_to_binaryto tolerate malformed input and return NULL instead.- Valid Base64 string should include symbols from in base64 alphabet (A-Za-z0-9+/), optional padding (=), and optional whitespaces. Whitespaces are skipped in conversion except when they are preceded by padding symbol(s). If padding is present it should conclude the string and follow rules described in RFC 4648 § 4.
- Valid hexadecimal strings should include only allowed symbols (0-9A-Fa-f).
- Valid values for fmtare case-insensitivehex,base64,utf-8,utf8.
 
- Valid Base64 string should include symbols from in base64 alphabet (A-Za-z0-9+/), optional padding (
- Since Spark 3.4, Spark throws only PartitionsAlreadyExistExceptionwhen it creates partitions but some of them exist already. In Spark 3.3 or earlier, Spark can throw eitherPartitionsAlreadyExistExceptionorPartitionAlreadyExistsException.
- Since Spark 3.4, Spark will do validation for partition spec in ALTER PARTITION to follow the behavior of spark.sql.storeAssignmentPolicywhich may cause an exception if type conversion fails, e.g.ALTER TABLE .. ADD PARTITION(p='a')if columnpis int type. To restore the legacy behavior, setspark.sql.legacy.skipTypeValidationOnAlterPartitiontotrue.
- Since Spark 3.4, vectorized readers are enabled by default for the nested data types (array, map and struct). To restore the legacy behavior, set spark.sql.orc.enableNestedColumnVectorizedReaderandspark.sql.parquet.enableNestedColumnVectorizedReadertofalse.
- Since Spark 3.4, BinaryTypeis not supported in CSV datasource. In Spark 3.3 or earlier, users can write binary columns in CSV datasource, but the output content in CSV files isObject.toString()which is meaningless; meanwhile, if users read CSV tables with binary columns, Spark will throw anUnsupported type: binaryexception.
- Since Spark 3.4, bloom filter joins are enabled by default. To restore the legacy behavior, set spark.sql.optimizer.runtime.bloomFilter.enabledtofalse.
- Since Spark 3.4, when schema inference on external Parquet files, INT64 timestamps with annotation isAdjustedToUTC=falsewill be inferred as TimestampNTZ type instead of Timestamp type. To restore the legacy behavior, setspark.sql.parquet.inferTimestampNTZ.enabledtofalse.
Upgrading from Spark SQL 3.2 to 3.3
- 
    Since Spark 3.3, the histogram_numericfunction in Spark SQL returns an output type of an array of structs (x, y), where the type of the ‘x’ field in the return value is propagated from the input values consumed in the aggregate function. In Spark 3.2 or earlier, ‘x’ always had double type. Optionally, use the configurationspark.sql.legacy.histogramNumericPropagateInputTypesince Spark 3.3 to revert back to the previous behavior.
- 
    Since Spark 3.3, DayTimeIntervalTypein Spark SQL is mapped to Arrow’sDurationtype inArrowWriterandArrowColumnVectordeveloper APIs. Previously,DayTimeIntervalTypewas mapped to Arrow’sIntervaltype which does not match with the types of other languages Spark SQL maps. For example,DayTimeIntervalTypeis mapped tojava.time.Durationin Java.
- 
    Since Spark 3.3, the functions lpadandrpadhave been overloaded to support byte sequences. When the first argument is a byte sequence, the optional padding pattern must also be a byte sequence and the result is a BINARY value. The default padding pattern in this case is the zero byte. To restore the legacy behavior of always returning string types, setspark.sql.legacy.lpadRpadAlwaysReturnStringtotrue.
- 
    Since Spark 3.3, Spark turns a non-nullable schema into nullable for API DataFrameReader.schema(schema: StructType).json(jsonDataset: Dataset[String])andDataFrameReader.schema(schema: StructType).csv(csvDataset: Dataset[String])when the schema is specified by the user and contains non-nullable fields. To restore the legacy behavior of respecting the nullability, setspark.sql.legacy.respectNullabilityInTextDatasetConversiontotrue.
- 
    Since Spark 3.3, when the date or timestamp pattern is not specified, Spark converts an input string to a date/timestamp using the CASTexpression approach. The changes affect CSV/JSON datasources and parsing of partition values. In Spark 3.2 or earlier, when the date or timestamp pattern is not set, Spark uses the default patterns:yyyy-MM-ddfor dates andyyyy-MM-dd HH:mm:ssfor timestamps. After the changes, Spark still recognizes the pattern together withDate patterns: - [+-]yyyy*
- [+-]yyyy*-[m]m
- [+-]yyyy*-[m]m-[d]d
- [+-]yyyy*-[m]m-[d]d
- [+-]yyyy*-[m]m-[d]d *
- [+-]yyyy*-[m]m-[d]dT*
 Timestamp patterns: - [+-]yyyy*
- [+-]yyyy*-[m]m
- [+-]yyyy*-[m]m-[d]d
- [+-]yyyy*-[m]m-[d]d
- [+-]yyyy*-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]
- [+-]yyyy*-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]
- [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]
- T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]
 
- 
    Since Spark 3.3, the strfmtinformat_string(strfmt, obj, ...)andprintf(strfmt, obj, ...)will no longer support to use “0$” to specify the first argument, the first argument should always reference by “1$” when use argument index to indicating the position of the argument in the argument list.
- 
    Since Spark 3.3, nulls are written as empty strings in CSV data source by default. In Spark 3.2 or earlier, nulls were written as empty strings as quoted empty strings, "". To restore the previous behavior, setnullValueto"", or set the configurationspark.sql.legacy.nullValueWrittenAsQuotedEmptyStringCsvtotrue.
- 
    Since Spark 3.3, DESCRIBE FUNCTION fails if the function does not exist. In Spark 3.2 or earlier, DESCRIBE FUNCTION can still run and print “Function: func_name not found”. 
- 
    Since Spark 3.3, the table property externalbecomes reserved. Certain commands will fail if you specify theexternalproperty, such asCREATE TABLE ... TBLPROPERTIESandALTER TABLE ... SET TBLPROPERTIES. In Spark 3.2 and earlier, the table propertyexternalis silently ignored. You can setspark.sql.legacy.notReservePropertiestotrueto restore the old behavior.
- 
    Since Spark 3.3, DROP FUNCTION fails if the function name matches one of the built-in functions’ name and is not qualified. In Spark 3.2 or earlier, DROP FUNCTION can still drop a persistent function even if the name is not qualified and is the same as a built-in function’s name. 
- 
    Since Spark 3.3, when reading values from a JSON attribute defined as FloatTypeorDoubleType, the strings"+Infinity","+INF", and"-INF"are now parsed to the appropriate values, in addition to the already supported"Infinity"and"-Infinity"variations. This change was made to improve consistency with Jackson’s parsing of the unquoted versions of these values. Also, theallowNonNumericNumbersoption is now respected so these strings will now be considered invalid if this option is disabled.
- 
    Since Spark 3.3, Spark will try to use built-in data source writer instead of Hive serde in INSERT OVERWRITE DIRECTORY. This behavior is effective only ifspark.sql.hive.convertMetastoreParquetorspark.sql.hive.convertMetastoreOrcis enabled respectively for Parquet and ORC formats. To restore the behavior before Spark 3.3, you can setspark.sql.hive.convertMetastoreInsertDirtofalse.
- 
    Since Spark 3.3, the precision of the return type of round-like functions has been fixed. This may cause Spark throw AnalysisExceptionof theCANNOT_UP_CAST_DATATYPEerror class when using views created by prior versions. In such cases, you need to recreate the views using ALTER VIEW AS or CREATE OR REPLACE VIEW AS with newer Spark versions.
- 
    Since Spark 3.3, the unbase64function throws error for a malformedstrinput. Usetry_to_binary(<str>, 'base64')to tolerate malformed input and return NULL instead. In Spark 3.2 and earlier, theunbase64function returns a best-efforts result for a malformedstrinput.
- 
    Since Spark 3.3, when reading Parquet files that were not produced by Spark, Parquet timestamp columns with annotation isAdjustedToUTC = falseare inferred as TIMESTAMP_NTZ type during schema inference. In Spark 3.2 and earlier, these columns are inferred as TIMESTAMP type. To restore the behavior before Spark 3.3, you can setspark.sql.parquet.inferTimestampNTZ.enabledtofalse.
- 
    Since Spark 3.3.1 and 3.2.3, for SELECT ... GROUP BY a GROUPING SETS (b)-style SQL statements,grouping__idreturns different values from Apache Spark 3.2.0, 3.2.1, 3.2.2, and 3.3.0. It computes based on user-given group-by expressions plus grouping set columns. To restore the behavior before 3.3.1 and 3.2.3, you can setspark.sql.legacy.groupingIdWithAppendedUserGroupBy. For details, see SPARK-40218 and SPARK-40562.
Upgrading from Spark SQL 3.1 to 3.2
- 
    Since Spark 3.2, ADD FILE/JAR/ARCHIVE commands require each path to be enclosed by "or'if the path contains whitespaces.
- 
    Since Spark 3.2, all the supported JDBC dialects use StringType for ROWID. In Spark 3.1 or earlier, Oracle dialect uses StringType and the other dialects use LongType. 
- 
    In Spark 3.2, PostgreSQL JDBC dialect uses StringType for MONEY and MONEY[] is not supported due to the JDBC driver for PostgreSQL can’t handle those types properly. In Spark 3.1 or earlier, DoubleType and ArrayType of DoubleType are used respectively. 
- 
    In Spark 3.2, spark.sql.adaptive.enabledis enabled by default. To restore the behavior before Spark 3.2, you can setspark.sql.adaptive.enabledtofalse.
- In Spark 3.2, the following meta-characters are escaped in the show()action. In Spark 3.1 or earlier, the following metacharacters are output as it is.- \n(new line)
- \r(carrige ret)
- \t(horizontal tab)
- \f(form feed)
- \b(backspace)
- \u000B(vertical tab)
- \u0007(bell)
 
- 
    In Spark 3.2, ALTER TABLE .. RENAME TO PARTITIONthrowsPartitionAlreadyExistsExceptioninstead ofAnalysisExceptionfor tables from Hive external when the target partition already exists.
- 
    In Spark 3.2, script transform default FIELD DELIMIT is \u0001for no serde mode, serde propertyfield.delimis\tfor Hive serde mode when user specifies serde. In Spark 3.1 or earlier, the default FIELD DELIMIT is\t, serde propertyfield.delimis\u0001for Hive serde mode when user specifies serde.
- 
    In Spark 3.2, the auto-generated Cast(such as those added by type coercion rules) will be stripped when generating column alias names. E.g.,sql("SELECT floor(1)").columnswill beFLOOR(1)instead ofFLOOR(CAST(1 AS DOUBLE)).
- 
    In Spark 3.2, the output schema of SHOW TABLESbecomesnamespace: string, tableName: string, isTemporary: boolean. In Spark 3.1 or earlier, thenamespacefield was nameddatabasefor the builtin catalog, and there is noisTemporaryfield for v2 catalogs. To restore the old schema with the builtin catalog, you can setspark.sql.legacy.keepCommandOutputSchematotrue.
- 
    In Spark 3.2, the output schema of SHOW TABLE EXTENDEDbecomesnamespace: string, tableName: string, isTemporary: boolean, information: string. In Spark 3.1 or earlier, thenamespacefield was nameddatabasefor the builtin catalog, and no change for the v2 catalogs. To restore the old schema with the builtin catalog, you can setspark.sql.legacy.keepCommandOutputSchematotrue.
- 
    In Spark 3.2, the output schema of SHOW TBLPROPERTIESbecomeskey: string, value: stringwhether you specify the table property key or not. In Spark 3.1 and earlier, the output schema ofSHOW TBLPROPERTIESisvalue: stringwhen you specify the table property key. To restore the old schema with the builtin catalog, you can setspark.sql.legacy.keepCommandOutputSchematotrue.
- 
    In Spark 3.2, the output schema of DESCRIBE NAMESPACEbecomesinfo_name: string, info_value: string. In Spark 3.1 or earlier, theinfo_namefield was nameddatabase_description_itemand theinfo_valuefield was nameddatabase_description_valuefor the builtin catalog. To restore the old schema with the builtin catalog, you can setspark.sql.legacy.keepCommandOutputSchematotrue.
- In Spark 3.2, table refreshing clears cached data of the table as well as of all its dependents such as views while keeping the dependents cached. The following commands perform table refreshing:
    - ALTER TABLE .. ADD PARTITION
- ALTER TABLE .. RENAME PARTITION
- ALTER TABLE .. DROP PARTITION
- ALTER TABLE .. RECOVER PARTITIONS
- MSCK REPAIR TABLE
- LOAD DATA
- REFRESH TABLE
- TRUNCATE TABLE
- and the method spark.catalog.refreshTableIn Spark 3.1 and earlier, table refreshing leaves dependents uncached.
 
- 
    In Spark 3.2, the usage of count(tblName.*)is blocked to avoid producing ambiguous results. Becausecount(*)andcount(tblName.*)will output differently if there is any null values. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.allowStarWithSingleTableIdentifierInCounttotrue.
- 
    In Spark 3.2, we support typed literals in the partition spec of INSERT and ADD/DROP/RENAME PARTITION. For example, ADD PARTITION(dt = date'2020-01-01')adds a partition with date value2020-01-01. In Spark 3.1 and earlier, the partition value will be parsed as string valuedate '2020-01-01', which is an illegal date value, and we add a partition with null value at the end.
- 
    In Spark 3.2, DataFrameNaFunctions.replace()no longer uses exact string match for the input column names, to match the SQL syntax and support qualified column names. Input column name having a dot in the name (not nested) needs to be escaped with backtick `. Now, it throwsAnalysisExceptionif the column is not found in the data frame schema. It also throwsIllegalArgumentExceptionif the input column name is a nested column. In Spark 3.1 and earlier, it used to ignore invalid input column name and nested column name.
- 
    In Spark 3.2, the dates subtraction expression such as date1 - date2returns values ofDayTimeIntervalType. In Spark 3.1 and earlier, the returned type isCalendarIntervalType. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.interval.enabledtotrue.
- 
    In Spark 3.2, the timestamps subtraction expression such as timestamp '2021-03-31 23:48:00' - timestamp '2021-01-01 00:00:00'returns values ofDayTimeIntervalType. In Spark 3.1 and earlier, the type of the same expression isCalendarIntervalType. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.interval.enabledtotrue.
- 
    In Spark 3.2, CREATE TABLE .. LIKE ..command can not use reserved properties. You need their specific clauses to specify them, for example,CREATE TABLE test1 LIKE test LOCATION 'some path'. You can setspark.sql.legacy.notReservePropertiestotrueto ignore theParseException, in this case, these properties will be silently removed, for example:TBLPROPERTIES('owner'='yao')will have no effect. In Spark version 3.1 and below, the reserved properties can be used inCREATE TABLE .. LIKE ..command but have no side effects, for example,TBLPROPERTIES('location'='/tmp')does not change the location of the table but only create a headless property just like'a'='b'.
- 
    In Spark 3.2, TRANSFORMoperator can’t support alias in inputs. In Spark 3.1 and earlier, we can write script transform likeSELECT TRANSFORM(a AS c1, b AS c2) USING 'cat' FROM TBL.
- 
    In Spark 3.2, TRANSFORMoperator can supportArrayType/MapType/StructTypewithout Hive SerDe, in this mode, we useStructsToJosnto convertArrayType/MapType/StructTypecolumn toSTRINGand useJsonToStructsto parseSTRINGtoArrayType/MapType/StructType. In Spark 3.1, Spark just support caseArrayType/MapType/StructTypecolumn asSTRINGbut can’t support parseSTRINGtoArrayType/MapType/StructTypeoutput columns.
- 
    In Spark 3.2, the unit-to-unit interval literals like INTERVAL '1-1' YEAR TO MONTHand the unit list interval literals likeINTERVAL '3' DAYS '1' HOURare converted to ANSI interval types:YearMonthIntervalTypeorDayTimeIntervalType. In Spark 3.1 and earlier, such interval literals are converted toCalendarIntervalType. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.interval.enabledtotrue.
- 
    In Spark 3.2, the unit list interval literals can not mix year-month fields (YEAR and MONTH) and day-time fields (WEEK, DAY, …, MICROSECOND). For example, INTERVAL 1 month 1 houris invalid in Spark 3.2. In Spark 3.1 and earlier, there is no such limitation and the literal returns value ofCalendarIntervalType. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.interval.enabledtotrue.
- 
    In Spark 3.2, Spark supports DayTimeIntervalTypeandYearMonthIntervalTypeas inputs and outputs ofTRANSFORMclause in HiveSERDEmode, the behavior is different between HiveSERDEmode andROW FORMAT DELIMITEDmode when these two types are used as inputs. In HiveSERDEmode,DayTimeIntervalTypecolumn is converted toHiveIntervalDayTime, its string format is[-]?d h:m:s.n, but inROW FORMAT DELIMITEDmode the format isINTERVAL '[-]?d h:m:s.n' DAY TO TIME. In HiveSERDEmode,YearMonthIntervalTypecolumn is converted toHiveIntervalYearMonth, its string format is[-]?y-m, but inROW FORMAT DELIMITEDmode the format isINTERVAL '[-]?y-m' YEAR TO MONTH.
- 
    In Spark 3.2, hash(0) == hash(-0)for floating point types. Previously, different values were generated.
- 
    In Spark 3.2, CREATE TABLE AS SELECTwith non-emptyLOCATIONwill throwAnalysisException. To restore the behavior before Spark 3.2, you can setspark.sql.legacy.allowNonEmptyLocationInCTAStotrue.
- 
    In Spark 3.2, special datetime values such as epoch,today,yesterday,tomorrow, andnoware supported in typed literals or in cast of foldable strings only, for instance,select timestamp'now'orselect cast('today' as date). In Spark 3.1 and 3.0, such special values are supported in any casts of strings to dates/timestamps. To keep these special values as dates/timestamps in Spark 3.1 and 3.0, you should replace them manually, e.g.if (c in ('now', 'today'), current_date(), cast(c as date)).
- 
    In Spark 3.2, FloatTypeis mapped toFLOATin MySQL. Prior to this, it used to be mapped toREAL, which is by default a synonym toDOUBLE PRECISIONin MySQL.
- 
    In Spark 3.2, the query executions triggered by DataFrameWriterare always namedcommandwhen being sent toQueryExecutionListener. In Spark 3.1 and earlier, the name is one ofsave,insertInto,saveAsTable.
- 
    In Spark 3.2, Dataset.unionByNamewithallowMissingColumnsset to true will add missing nested fields to the end of structs. In Spark 3.1, nested struct fields are sorted alphabetically.
- 
    In Spark 3.2, create/alter view will fail if the input query output columns contain auto-generated alias. This is necessary to make sure the query output column names are stable across different spark versions. To restore the behavior before Spark 3.2, set spark.sql.legacy.allowAutoGeneratedAliasForViewtotrue.
- In Spark 3.2, date +/- interval with only day-time fields such as date '2011-11-11' + interval 12 hoursreturns timestamp. In Spark 3.1 and earlier, the same expression returns date. To restore the behavior before Spark 3.2, you can usecastto convert timestamp as date.
Upgrading from Spark SQL 3.0 to 3.1
- 
    In Spark 3.1, statistical aggregation function includes std,stddev,stddev_samp,variance,var_samp,skewness,kurtosis,covar_samp,corrwill returnNULLinstead ofDouble.NaNwhenDivideByZerooccurs during expression evaluation, for example, whenstddev_sampapplied on a single element set. In Spark version 3.0 and earlier, it will returnDouble.NaNin such case. To restore the behavior before Spark 3.1, you can setspark.sql.legacy.statisticalAggregatetotrue.
- 
    In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set spark.sql.legacy.integerGroupingIdtotrue.
- 
    In Spark 3.1, SQL UI data adopts the formattedmode for the query plan explain results. To restore the behavior before Spark 3.1, you can setspark.sql.ui.explainModetoextended.
- 
    In Spark 3.1, from_unixtime,unix_timestamp,to_unix_timestamp,to_timestampandto_datewill fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they resultNULL.
- 
    In Spark 3.1, the Parquet, ORC, Avro and JSON datasources throw the exception org.apache.spark.sql.AnalysisException: Found duplicate column(s) in the data schemain read if they detect duplicate names in top-level columns as well in nested structures. The datasources take into account the SQL configspark.sql.caseSensitivewhile detecting column name duplicates.
- 
    In Spark 3.1, structs and maps are wrapped by the {}brackets in casting them to strings. For instance, theshow()action and theCASTexpression use such brackets. In Spark 3.0 and earlier, the[]brackets are used for the same purpose. To restore the behavior before Spark 3.1, you can setspark.sql.legacy.castComplexTypesToString.enabledtotrue.
- 
    In Spark 3.1, NULL elements of structures, arrays and maps are converted to “null” in casting them to strings. In Spark 3.0 or earlier, NULL elements are converted to empty strings. To restore the behavior before Spark 3.1, you can set spark.sql.legacy.castComplexTypesToString.enabledtotrue.
- 
    In Spark 3.1, when spark.sql.ansi.enabledis false, Spark always returns null if the sum of decimal type column overflows. In Spark 3.0 or earlier, in the case, the sum of decimal type column may return null or incorrect result, or even fails at runtime (depending on the actual query plan execution).
- 
    In Spark 3.1, pathoption cannot coexist when the following methods are called with path parameter(s):DataFrameReader.load(),DataFrameWriter.save(),DataStreamReader.load(), orDataStreamWriter.start(). In addition,pathsoption cannot coexist forDataFrameReader.load(). For example,spark.read.format("csv").option("path", "/tmp").load("/tmp2")orspark.read.option("path", "/tmp").csv("/tmp2")will throworg.apache.spark.sql.AnalysisException. In Spark version 3.0 and below,pathoption is overwritten if one path parameter is passed to above methods;pathoption is added to the overall paths if multiple path parameters are passed toDataFrameReader.load(). To restore the behavior before Spark 3.1, you can setspark.sql.legacy.pathOptionBehavior.enabledtotrue.
- 
    In Spark 3.1, IllegalArgumentExceptionis returned for the incomplete interval literals, e.g.INTERVAL '1',INTERVAL '1 DAY 2', which are invalid. In Spark 3.0, these literals result inNULLs.
- 
    In Spark 3.1, we remove the built-in Hive 1.2. You need to migrate your custom SerDes to Hive 2.3. See HIVE-15167 for more details. 
- 
    In Spark 3.1, loading and saving of timestamps from/to parquet files fails if the timestamps are before 1900-01-01 00:00:00Z, and loaded (saved) as the INT96 type. In Spark 3.0, the actions don’t fail but might lead to shifting of the input timestamps due to rebasing from/to Julian to/from Proleptic Gregorian calendar. To restore the behavior before Spark 3.1, you can set spark.sql.legacy.parquet.int96RebaseModeInReador/andspark.sql.legacy.parquet.int96RebaseModeInWritetoLEGACY.
- 
    In Spark 3.1, the schema_of_jsonandschema_of_csvfunctions return the schema in the SQL format in which field names are quoted. In Spark 3.0, the function returns a catalog string without field quoting and in lower case.
- 
    In Spark 3.1, refreshing a table will trigger an uncache operation for all other caches that reference the table, even if the table itself is not cached. In Spark 3.0 the operation will only be triggered if the table itself is cached. 
- 
    In Spark 3.1, creating or altering a permanent view will capture runtime SQL configs and store them as view properties. These configs will be applied during the parsing and analysis phases of the view resolution. To restore the behavior before Spark 3.1, you can set spark.sql.legacy.useCurrentConfigsForViewtotrue.
- 
    In Spark 3.1, the temporary view will have same behaviors with the permanent view, i.e. capture and store runtime SQL configs, SQL text, catalog and namespace. The captured view properties will be applied during the parsing and analysis phases of the view resolution. To restore the behavior before Spark 3.1, you can set spark.sql.legacy.storeAnalyzedPlanForViewtotrue.
- 
    In Spark 3.1, temporary view created via CACHE TABLE ... AS SELECTwill also have the same behavior with permanent view. In particular, when the temporary view is dropped, Spark will invalidate all its cache dependents, as well as the cache for the temporary view itself. This is different from Spark 3.0 and below, which only does the latter. To restore the previous behavior, you can setspark.sql.legacy.storeAnalyzedPlanForViewtotrue.
- 
    Since Spark 3.1, CHAR/CHARACTER and VARCHAR types are supported in the table schema. Table scan/insertion will respect the char/varchar semantic. If char/varchar is used in places other than table schema, an exception will be thrown (CAST is an exception that simply treats char/varchar as string like before). To restore the behavior before Spark 3.1, which treats them as STRING types and ignores a length parameter, e.g. CHAR(4), you can setspark.sql.legacy.charVarcharAsStringtotrue.
- 
    In Spark 3.1, AnalysisExceptionis replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:- ALTER TABLE .. ADD PARTITIONthrows- PartitionsAlreadyExistExceptionif new partition exists already
- ALTER TABLE .. DROP PARTITIONthrows- NoSuchPartitionsExceptionfor not existing partitions
 
Upgrading from Spark SQL 3.0.1 to 3.0.2
- In Spark 3.0.2, AnalysisExceptionis replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:- ALTER TABLE .. ADD PARTITIONthrows- PartitionsAlreadyExistExceptionif new partition exists already
- ALTER TABLE .. DROP PARTITIONthrows- NoSuchPartitionsExceptionfor not existing partitions
 
- 
    In Spark 3.0.2, PARTITION(col=null)is always parsed as a null literal in the partition spec. In Spark 3.0.1 or earlier, it is parsed as a string literal of its text representation, e.g., string “null”, if the partition column is string type. To restore the legacy behavior, you can setspark.sql.legacy.parseNullPartitionSpecAsStringLiteralas true.
- In Spark 3.0.2, the output schema of SHOW DATABASESbecomesnamespace: string. In Spark version 3.0.1 and earlier, the schema wasdatabaseName: string. Since Spark 3.0.2, you can restore the old schema by settingspark.sql.legacy.keepCommandOutputSchematotrue.
Upgrading from Spark SQL 3.0 to 3.0.1
- 
    In Spark 3.0, JSON datasource and JSON function schema_of_jsoninfer TimestampType from string values if they match to the pattern defined by the JSON optiontimestampFormat. Since version 3.0.1, the timestamp type inference is disabled by default. Set the JSON optioninferTimestamptotrueto enable such type inference.
- 
    In Spark 3.0, when casting string to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing characters (<= ASCII 32) will be trimmed. For example, cast('\b1\b' as int)results1. Since Spark 3.0.1, only the leading and trailing whitespace ASCII characters will be trimmed. For example,cast('\t1\t' as int)results1butcast('\b1\b' as int)resultsNULL.
Upgrading from Spark SQL 2.4 to 3.0
Dataset/DataFrame APIs
- 
    In Spark 3.0, the Dataset and DataFrame API unionAllis no longer deprecated. It is an alias forunion.
- 
    In Spark 2.4 and below, Dataset.groupByKeyresults to a grouped dataset with key attribute is wrongly named as “value”, if the key is non-struct type, for example, int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries unexpected. For example, the schema ofds.groupByKey(...).count()is(value, count). Since Spark 3.0, we name the grouping attribute to “key”. The old behavior is preserved under a newly added configurationspark.sql.legacy.dataset.nameNonStructGroupingKeyAsValuewith a default value offalse.
- 
    In Spark 3.0, the column metadata will always be propagated in the API Column.nameandColumn.as. In Spark version 2.4 and earlier, the metadata ofNamedExpressionis set as theexplicitMetadatafor the new column at the time the API is called, it won’t change even if the underlyingNamedExpressionchanges metadata. To restore the behavior before Spark 3.0, you can use the APIas(alias: String, metadata: Metadata)with explicit metadata.
DDL Statements
- 
    In Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting stringtointanddoubletobooleanare disallowed. A runtime exception is thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and below, type conversions during table insertion are allowed as long as they are validCast. When inserting an out-of-range value to an integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the optionspark.sql.storeAssignmentPolicy, with a default value as “ANSI”. Setting the option as “Legacy” restores the previous behavior.
- 
    The ADD JARcommand previously returned a result set with the single value 0. It now returns an empty result set.
- 
    Spark 2.4 and below: the SETcommand works without any warnings even if the specified key is forSparkConfentries and it has no effect because the command does not updateSparkConf, but the behavior might confuse users. In 3.0, the command fails if aSparkConfkey is used. You can disable such a check by settingspark.sql.legacy.setCommandRejectsSparkCoreConfstofalse.
- 
    Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and below, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. In Spark 3.0, cache name and storage level are first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. 
- 
    In Spark 3.0, the properties listing below become reserved; commands fail if you specify reserved properties in places like CREATE DATABASE ... WITH DBPROPERTIESandALTER TABLE ... SET TBLPROPERTIES. You need their specific clauses to specify them, for example,CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'. You can setspark.sql.legacy.notReservePropertiestotrueto ignore theParseException, in this case, these properties will be silently removed, for example:SET DBPROPERTIES('location'='/tmp')will have no effect. In Spark version 2.4 and below, these properties are neither reserved nor have side effects, for example,SET DBPROPERTIES('location'='/tmp')do not change the location of the database but only create a headless property just like'a'='b'.Property (case sensitive) Database Reserved Table Reserved Remarks provider no yes For tables, use the USINGclause to specify it. Once set, it can’t be changed.location yes yes For databases and tables, use the LOCATIONclause to specify it.owner yes yes For databases and tables, it is determined by the user who runs spark and create the table. 
- 
    In Spark 3.0, you can use ADD FILEto add file directories as well. Earlier you could add only single files using this command. To restore the behavior of earlier versions, setspark.sql.legacy.addSingleFileInAddFiletotrue.
- 
    In Spark 3.0, SHOW TBLPROPERTIESthrowsAnalysisExceptionif the table does not exist. In Spark version 2.4 and below, this scenario causedNoSuchTableException.
- 
    In Spark 3.0, SHOW CREATE TABLE table_identifieralways returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, useSHOW CREATE TABLE table_identifier AS SERDEcommand instead.
- 
    In Spark 3.0, column of CHAR type is not allowed in non-Hive-Serde tables, and CREATE/ALTER TABLE commands will fail if CHAR type is detected. Please use STRING type instead. In Spark version 2.4 and below, CHAR type is treated as STRING type and the length parameter is simply ignored. 
UDFs and Built-in Functions
- 
    In Spark 3.0, the date_addanddate_subfunctions accepts only int, smallint, tinyint as the 2nd argument; fractional and non-literal strings are not valid anymore, for example:date_add(cast('1964-05-23' as date), '12.34')causesAnalysisException. Note that, string literals are still allowed, but Spark will throwAnalysisExceptionif the string content is not a valid integer. In Spark version 2.4 and below, if the 2nd argument is fractional or string value, it is coerced to int value, and the result is a date value of1964-06-04.
- 
    In Spark 3.0, the function percentile_approxand its aliasapprox_percentileonly accept integral value with range in[1, 2147483647]as its 3rd argumentaccuracy, fractional and string types are disallowed, for example,percentile_approx(10.0, 0.2, 1.8D)causesAnalysisException. In Spark version 2.4 and below, ifaccuracyis fractional or string value, it is coerced to an int value,percentile_approx(10.0, 0.2, 1.8D)is operated aspercentile_approx(10.0, 0.2, 1)which results in10.0.
- 
    In Spark 3.0, an analysis exception is thrown when hash expressions are applied on elements of MapType. To restore the behavior before Spark 3.0, setspark.sql.legacy.allowHashOnMapTypetotrue.
- 
    In Spark 3.0, when the array/mapfunction is called without any parameters, it returns an empty collection withNullTypeas element type. In Spark version 2.4 and below, it returns an empty collection withStringTypeas element type. To restore the behavior before Spark 3.0, you can setspark.sql.legacy.createEmptyCollectionUsingStringTypetotrue.
- 
    In Spark 3.0, the from_jsonfunctions supports two modes -PERMISSIVEandFAILFAST. The modes can be set via themodeoption. The default mode becamePERMISSIVE. In previous versions, behavior offrom_jsondid not conform to eitherPERMISSIVEnorFAILFAST, especially in processing of malformed JSON records. For example, the JSON string{"a" 1}with the schemaa INTis converted tonullby previous versions but Spark 3.0 converts it toRow(null).
- 
    In Spark version 2.4 and below, you can create map values with map type key via built-in function such as CreateMap,MapFromArrays, etc. In Spark 3.0, it’s not allowed to create map values with map type key with these built-in functions. Users can usemap_entriesfunction to convert map to array<struct<key, value» as a workaround. In addition, users can still read map values with map type key from data source or Java/Scala collections, though it is discouraged.
- 
    In Spark version 2.4 and below, you can create a map with duplicated keys via built-in functions like CreateMap,StringToMap, etc. The behavior of map with duplicated keys is undefined, for example, map look up respects the duplicated key appears first,Dataset.collectonly keeps the duplicated key appears last,MapKeysreturns duplicated keys, etc. In Spark 3.0, Spark throwsRuntimeExceptionwhen duplicated keys are found. You can setspark.sql.mapKeyDedupPolicytoLAST_WINto deduplicate map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (for example, Parquet), the behavior is undefined.
- 
    In Spark 3.0, using org.apache.spark.sql.functions.udf(AnyRef, DataType)is not allowed by default. Remove the return type parameter to automatically switch to typed Scala udf is recommended, or setspark.sql.legacy.allowUntypedScalaUDFto true to keep using it. In Spark version 2.4 and below, iforg.apache.spark.sql.functions.udf(AnyRef, DataType)gets a Scala closure with primitive-type argument, the returned UDF returns null if the input values is null. However, in Spark 3.0, the UDF returns the default value of the Java type if the input value is null. For example,val f = udf((x: Int) => x, IntegerType),f($"x")returns null in Spark 2.4 and below if columnxis null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default.
- 
    In Spark 3.0, a higher-order function existsfollows the three-valued boolean logic, that is, if thepredicatereturns anynulls and notrueis obtained, thenexistsreturnsnullinstead offalse. For example,exists(array(1, null, 3), x -> x % 2 == 0)isnull. The previous behavior can be restored by settingspark.sql.legacy.followThreeValuedLogicInArrayExiststofalse.
- 
    In Spark 3.0, the add_monthsfunction does not adjust the resulting date to a last day of month if the original date is a last day of months. For example,select add_months(DATE'2019-02-28', 1)results2019-03-28. In Spark version 2.4 and below, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to2019-02-28results in2019-03-31.
- 
    In Spark version 2.4 and below, the current_timestampfunction returns a timestamp with millisecond resolution only. In Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution.
- 
    In Spark 3.0, a 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and below, the 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. 
- 
    The result of java.lang.Math’slog,log1p,exp,expm1, andpowmay vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions likeLOG10) return values consistent withjava.lang.StrictMath. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly matchjava.lang.Mathon x86 platforms in cases like, for example,log(3.0), whose value varies betweenMath.log()andStrictMath.log().
- 
    In Spark 3.0, the castfunction processes string literals such as ‘Infinity’, ‘+Infinity’, ‘-Infinity’, ‘NaN’, ‘Inf’, ‘+Inf’, ‘-Inf’ in a case-insensitive manner when casting the literals toDoubleorFloattype to ensure greater compatibility with other database systems. This behavior change is illustrated in the table below:Operation Result before Spark 3.0 Result in Spark 3.0 CAST(‘infinity’ AS DOUBLE) NULL Double.PositiveInfinity CAST(‘+infinity’ AS DOUBLE) NULL Double.PositiveInfinity CAST(‘inf’ AS DOUBLE) NULL Double.PositiveInfinity CAST(‘inf’ AS DOUBLE) NULL Double.PositiveInfinity CAST(‘-infinity’ AS DOUBLE) NULL Double.NegativeInfinity CAST(‘-inf’ AS DOUBLE) NULL Double.NegativeInfinity CAST(‘infinity’ AS FLOAT) NULL Float.PositiveInfinity CAST(‘+infinity’ AS FLOAT) NULL Float.PositiveInfinity CAST(‘inf’ AS FLOAT) NULL Float.PositiveInfinity CAST(‘+inf’ AS FLOAT) NULL Float.PositiveInfinity CAST(‘-infinity’ AS FLOAT) NULL Float.NegativeInfinity CAST(‘-inf’ AS FLOAT) NULL Float.NegativeInfinity CAST(‘nan’ AS DOUBLE) NULL Double.NaN CAST(‘nan’ AS FLOAT) NULL Float.NaN 
- 
    In Spark 3.0, when casting interval values to string type, there is no “interval” prefix, for example, 1 days 2 hours. In Spark version 2.4 and below, the string contains the “interval” prefix likeinterval 1 days 2 hours.
- 
    In Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, for example, cast(' 1\t' as int)results1,cast(' 1\t' as boolean)resultstrue,cast('2019-10-10\t as date)results the date value2019-10-10. In Spark version 2.4 and below, when casting string to integrals and booleans, it does not trim the whitespaces from both ends; the foregoing results isnull, while to datetimes, only the trailing spaces (= ASCII 32) are removed.
Query Engine
- 
    In Spark version 2.4 and below, SQL queries such as FROM <table>orFROM <table> UNION ALL FROM <table>are supported by accident. In hive-styleFROM <table> SELECT <expr>, theSELECTclause is not negligible. Neither Hive nor Presto support this syntax. These queries are treated as invalid in Spark 3.0.
- 
    In Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'throws parser exception.
- 
    In Spark 3.0, numbers written in scientific notation(for example, 1E2) would be parsed as Double. In Spark version 2.4 and below, they’re parsed as Decimal. To restore the behavior before Spark 3.0, you can setspark.sql.legacy.exponentLiteralAsDecimal.enabledtotrue.
- 
    In Spark 3.0, day-time interval strings are converted to intervals with respect to the fromandtobounds. If an input string does not match to the pattern defined by specified bounds, theParseExceptionexception is thrown. For example,interval '2 10:20' hour to minuteraises the exception because the expected format is[+|-]h[h]:[m]m. In Spark version 2.4, thefrombound was not taken into account, and thetobound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted tointerval 10 hours 20 minutes. To restore the behavior before Spark 3.0, you can setspark.sql.legacy.fromDayTimeString.enabledtotrue.
- 
    In Spark 3.0, negative scale of decimal is not allowed by default, for example, data type of literal like 1E10BDisDecimalType(11, 0). In Spark version 2.4 and below, it wasDecimalType(2, -9). To restore the behavior before Spark 3.0, you can setspark.sql.legacy.allowNegativeScaleOfDecimaltotrue.
- 
    In Spark 3.0, the unary arithmetic operator plus( +) only accepts string, numeric and interval type values as inputs. Besides,+with an integral string representation is coerced to a double value, for example,+'1'returns1.0. In Spark version 2.4 and below, this operator is ignored. There is no type checking for it, thus, all type values with a+prefix are valid, for example,+ array(1, 2)is valid and results[1, 2]. Besides, there is no type coercion for it at all, for example, in Spark 2.4, the result of+'1'is string1.
- 
    In Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: val df1 = ...; val df2 = df1.filter(...);, thendf1.join(df2, df1("a") > df2("a"))returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, anddf1("a")is exactly the same asdf2("a")in Spark. To restore the behavior before Spark 3.0, you can setspark.sql.analyzer.failAmbiguousSelfJointofalse.
- 
    In Spark 3.0, spark.sql.legacy.ctePrecedencePolicyis introduced to control the behavior for name conflicting in the nested WITH clause. By default valueEXCEPTION, Spark throws an AnalysisException, it forces users to choose the specific substitution order they wanted. If set toCORRECTED(which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config tofalse,WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2returns2, while setting it toLEGACY, the result is1which is the behavior in version 2.4 and below.
- 
    In Spark 3.0, configuration spark.sql.crossJoin.enabledbecome internal configuration, and is true by default, so by default spark won’t raise exception on sql with implicit cross join.
- 
    In Spark version 2.4 and below, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys, and join keys. In Spark 3.0, this bug is fixed. For example, Seq(-0.0, 0.0).toDF("d").groupBy("d").count()returns[(0.0, 2)]in Spark 3.0, and[(0.0, 1), (-0.0, 1)]in Spark 2.4 and below.
- 
    In Spark version 2.4 and below, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. In Spark 3.0, such time zone ids are rejected, and Spark throws java.time.DateTimeException.
- 
    In Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and so on. Spark 3.0 uses Java 8 API classes from the java.timepackages that are based on ISO chronology. In Spark version 2.4 and below, those operations are performed using the hybrid calendar (Julian + Gregorian. The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API:- 
        Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the unix_timestamp,date_format,to_unix_timestamp,from_unixtime,to_date,to_timestampfunctions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in Datetime Patterns for Formatting and Parsing, which is implemented via DateTimeFormatter under the hood. New implementation performs strict checking of its input. For example, the2015-07-22 10:00:00timestamp cannot be parse if pattern isyyyy-MM-ddbecause the parser does not consume whole input. Another example is the31/01/2015 00:00input cannot be parsed by thedd/MM/yyyy hh:mmpattern becausehhsupposes hours in the range1-12. In Spark version 2.4 and below,java.text.SimpleDateFormatis used for timestamp/date string conversions, and the supported patterns are described in SimpleDateFormat. The old behavior can be restored by settingspark.sql.legacy.timeParserPolicytoLEGACY.
- 
        The weekofyear,weekday,dayofweek,date_trunc,from_utc_timestamp,to_utc_timestamp, andunix_timestampfunctions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone.
- 
        The JDBC options lowerBoundandupperBoundare converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL configspark.sql.session.timeZone. In Spark version 2.4 and below, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone.
- 
        Formatting TIMESTAMPandDATEliterals.
- 
        Creating typed TIMESTAMPandDATEliterals from strings. In Spark 3.0, string conversion to typedTIMESTAMP/DATEliterals is performed via casting toTIMESTAMP/DATEvalues. For example,TIMESTAMP '2019-12-23 12:59:30'is semantically equal toCAST('2019-12-23 12:59:30' AS TIMESTAMP). When the input string does not contain information about time zone, the time zone from the SQL configspark.sql.session.timeZoneis used in that case. In Spark version 2.4 and below, the conversion is based on JVM system time zone. The different sources of the default time zone may change the behavior of typedTIMESTAMPandDATEliterals.
 
- 
        
- 
    In Spark 3.0, TIMESTAMPliterals are converted to strings using the SQL configspark.sql.session.timeZone. In Spark version 2.4 and below, the conversion uses the default time zone of the Java virtual machine.
- 
    In Spark 3.0, Spark casts StringtoDate/Timestampin binary comparisons with dates/timestamps. The previous behavior of castingDate/TimestamptoStringcan be restored by settingspark.sql.legacy.typeCoercion.datetimeToString.enabledtotrue.
- In Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that are converted to ordinary date or timestamp values when read. The following string values are supported for dates:
    - epoch [zoneId]- 1970-01-01
- today [zoneId]- the current date in the time zone specified by- spark.sql.session.timeZone
- yesterday [zoneId]- the current date - 1
- tomorrow [zoneId]- the current date + 1
- now- the date of running the current query. It has the same notion as today
 For example SELECT date 'tomorrow' - date 'yesterday';should output2. Here are special timestamp values:- epoch [zoneId]- 1970-01-01 00:00:00+00 (Unix system time zero)
- today [zoneId]- midnight today
- yesterday [zoneId]- midnight yesterday
- tomorrow [zoneId]- midnight tomorrow
- now- current query start time
 For example SELECT timestamp 'tomorrow';.
- 
    Since Spark 3.0, when using EXTRACTexpression to extract the second field from date/timestamp values, the result will be aDecimalType(8, 6)value with 2 digits for second part, and 6 digits for the fractional part with microsecond precision. e.g.extract(second from to_timestamp('2019-09-20 10:10:10.1'))results10.100000. In Spark version 2.4 and earlier, it returns anIntegerTypevalue and the result for the former example is10.
- 
    In Spark 3.0, datetime pattern letter Fis aligned day of week in month that represents the concept of the count of days within the period of a week where the weeks are aligned to the start of the month. In Spark version 2.4 and earlier, it is week of month that represents the concept of the count of weeks within the month where weeks start on a fixed day-of-week, e.g.2020-07-30is 30 days (4 weeks and 2 days) after the first day of the month, sodate_format(date '2020-07-30', 'F')returns 2 in Spark 3.0, but as a week count in Spark 2.x, it returns 5 because it locates in the 5th week of July 2020, where week one is 2020-07-01 to 07-04.
- 
    In Spark 3.0, Spark will try to use built-in data source writer instead of Hive serde in CTAS. This behavior is effective only ifspark.sql.hive.convertMetastoreParquetorspark.sql.hive.convertMetastoreOrcis enabled respectively for Parquet and ORC formats. To restore the behavior before Spark 3.0, you can setspark.sql.hive.convertMetastoreCtastofalse.
- In Spark 3.0, Spark will try to use built-in data source writer instead of Hive serde to process inserting into partitioned ORC/Parquet tables created by using the HiveSQL syntax. This behavior is effective only if spark.sql.hive.convertMetastoreParquetorspark.sql.hive.convertMetastoreOrcis enabled respectively for Parquet and ORC formats. To restore the behavior before Spark 3.0, you can setspark.sql.hive.convertInsertingPartitionedTabletofalse.
Data Sources
- 
    In Spark version 2.4 and below, when reading a Hive SerDe table with Spark native data sources(parquet/orc), Spark infers the actual file schema and update the table schema in metastore. In Spark 3.0, Spark doesn’t infer the schema anymore. This should not cause any problems to end users, but if it does, set spark.sql.hive.caseSensitiveInferenceModetoINFER_AND_SAVE.
- 
    In Spark version 2.4 and below, partition column value is converted as null if it can’t be casted to corresponding user provided schema. In 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting spark.sql.sources.validatePartitionColumnstofalse.
- 
    In Spark 3.0, if files or subdirectories disappear during recursive directory listing (that is, they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless spark.sql.files.ignoreMissingFilesistrue(defaultfalse). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or duringREFRESH TABLE), not during query execution: the net change is thatspark.sql.files.ignoreMissingFilesis now obeyed during table file listing / query planning, not only at query execution time.
- 
    In Spark version 2.4 and below, the parser of JSON data source treats empty strings as null for some data types such as IntegerType. ForFloatType,DoubleType,DateTypeandTimestampType, it fails on empty strings and throws exceptions. Spark 3.0 disallows empty strings and will throw an exception for data types except forStringTypeandBinaryType. The previous behavior of allowing an empty string can be restored by settingspark.sql.legacy.json.allowEmptyString.enabledtotrue.
- 
    In Spark version 2.4 and below, JSON datasource and JSON functions like from_jsonconvert a bad JSON record to a row with allnulls in the PERMISSIVE mode when specified schema isStructType. In Spark 3.0, the returned row can contain non-nullfields if some of JSON column values were parsed and converted to desired types successfully.
- 
    In Spark 3.0, JSON datasource and JSON function schema_of_jsoninfer TimestampType from string values if they match to the pattern defined by the JSON optiontimestampFormat. Set JSON optioninferTimestamptofalseto disable such type inference.
- 
    In Spark version 2.4 and below, CSV datasource converts a malformed CSV string to a row with all nulls in the PERMISSIVE mode. In Spark 3.0, the returned row can contain non-nullfields if some of CSV column values were parsed and converted to desired types successfully.
- 
    In Spark 3.0, when Avro files are written with user provided schema, the fields are matched by field names between catalyst schema and Avro schema instead of positions. 
- 
    In Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark throws runtime NullPointerException if any of the records contains null. 
- 
    In Spark version 2.4 and below, CSV datasource can detect encoding of input files automatically when the files have BOM at the beginning. For instance, CSV datasource can recognize UTF-8, UTF-16BE, UTF-16LE, UTF-32BE and UTF-32LE in the multi-line mode (the CSV option multiLineis set totrue). In Spark 3.0, CSV datasource reads input files in encoding specified via the CSV optionencodingwhich has the default value of UTF-8. In this way, if file encoding doesn’t match to the encoding specified via the CSV option, Spark loads the file incorrectly. To solve the issue, users should either set correct encoding via the CSV optionencodingor set the option tonullwhich fallbacks to encoding auto-detection as in Spark versions before 3.0.
Others
- 
    In Spark version 2.4, when a Spark session is created via cloneSession(), the newly created Spark session inherits its configuration from its parentSparkContexteven though the same configuration may exist with a different value in its parent Spark session. In Spark 3.0, the configurations of a parentSparkSessionhave a higher precedence over the parentSparkContext. You can restore the old behavior by settingspark.sql.legacy.sessionInitWithConfigDefaultstotrue.
- 
    In Spark 3.0, if hive.default.fileformatis not found inSpark SQL configurationthen it falls back to thehive-site.xmlfile present in theHadoop configurationofSparkContext.
- 
    In Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for spark-sqlinterface, for example:Query Spark 2.4 Spark 3.0 SELECT CAST(1 AS decimal(38, 18));1 1.000000000000000000 
- 
    In Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3 and it brings following impacts: - 
        You may need to set spark.sql.hive.metastore.versionandspark.sql.hive.metastore.jarsaccording to the version of the Hive metastore you want to connect to. For example: setspark.sql.hive.metastore.versionto1.2.1andspark.sql.hive.metastore.jarstomavenif your Hive metastore version is 1.2.1.
- 
        You need to migrate your custom SerDes to Hive 2.3 or build your own Spark with hive-1.2profile. See HIVE-15167 for more details.
- 
        The decimal string representation can be different between Hive 1.2 and Hive 2.3 when using TRANSFORMoperator in SQL for script transformation, which depends on hive’s behavior. In Hive 1.2, the string representation omits trailing zeroes. But in Hive 2.3, it is always padded to 18 digits with trailing zeroes if necessary.
 
- 
        
Upgrading from Spark SQL 2.4.7 to 2.4.8
- In Spark 2.4.8, AnalysisExceptionis replaced by its sub-classes that are thrown for tables from Hive external catalog in the following situations:- ALTER TABLE .. ADD PARTITIONthrows- PartitionsAlreadyExistExceptionif new partition exists already
- ALTER TABLE .. DROP PARTITIONthrows- NoSuchPartitionsExceptionfor not existing partitions
 
Upgrading from Spark SQL 2.4.5 to 2.4.6
- In Spark 2.4.6, the RESETcommand does not reset the static SQL configuration values to the default. It only clears the runtime SQL configuration values.
Upgrading from Spark SQL 2.4.4 to 2.4.5
- 
    Since Spark 2.4.5, TRUNCATE TABLEcommand tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, setspark.sql.truncateTable.ignorePermissionAcl.enabledtotrue.
- 
    Since Spark 2.4.5, spark.sql.legacy.mssqlserver.numericMapping.enabledconfiguration is added in order to support the legacy MsSQLServer dialect mapping behavior using IntegerType and DoubleType for SMALLINT and REAL JDBC types, respectively. To restore the behaviour of 2.4.3 and earlier versions, setspark.sql.legacy.mssqlserver.numericMapping.enabledtotrue.
Upgrading from Spark SQL 2.4.3 to 2.4.4
- Since Spark 2.4.4, according to MsSqlServer Guide, MsSQLServer JDBC Dialect uses ShortType and FloatType for SMALLINT and REAL, respectively. Previously, IntegerType and DoubleType is used.
Upgrading from Spark SQL 2.4 to 2.4.1
- 
    The value of spark.executor.heartbeatInterval, when specified without units like “30” rather than “30s”, was inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. Unitless values are now consistently interpreted as milliseconds. Applications that set values like “30” need to specify a value with units like “30s” now, to avoid being interpreted as milliseconds; otherwise, the extremely short interval that results will likely cause applications to fail.
- 
    When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. Seq("str").toDS.as[Int]fails, butSeq("str").toDS.as[Boolean]works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e.Seq("str").toDS.as[Boolean]will fail during analysis. To restore the behavior before 2.4.1, setspark.sql.legacy.looseUpcasttotrue.
Upgrading from Spark SQL 2.3 to 2.4
- In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause array_containsfunction to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below.Query Spark 2.3 or Prior Spark 2.4 Remarks SELECT array_contains(array(1), 1.34D);truefalseIn Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. SELECT array_contains(array(1), '1');trueAnalysisExceptionis thrown.Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisExceptionis thrown since integer type can not be promoted to string type in a loss-less manner.SELECT array_contains(array(1), 'anystring');nullAnalysisExceptionis thrown.Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisExceptionis thrown since integer type can not be promoted to string type in a loss-less manner.
- 
    Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. For example, if ais astruct(a string, b int), in Spark 2.4a in (select (1 as a, 'a' as b) from range(1))is a valid query, whilea in (select 1, 'a' from range(1))is not. In previous version it was the opposite.
- 
    In versions 2.2.1+ and 2.3, if spark.sql.caseSensitiveis set to true, then theCURRENT_DATEandCURRENT_TIMESTAMPfunctions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive.
- 
    Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration spark.sql.legacy.setopsPrecedence.enabledwith a default value offalse. When this property is set totrue, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis.
- 
    Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. 
- 
    Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, spark.sql.orc.implandspark.sql.orc.filterPushdownchange their default values tonativeandtruerespectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Usespark.sql.orc.impl=hiveto create the files shared with Hive 2.1.1 and older.
- 
    Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. 
- 
    Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not UDF:f(col0 AS colA#28)butUDF:f(col0 AS `colA`).
- 
    Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. 
- 
    Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set falsetospark.sql.legacy.compareDateTimestampInTimestamprestores the previous behavior. This option will be removed in Spark 3.0.
- 
    Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set truetospark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocationrestores the previous behavior. This option will be removed in Spark 3.0.
- 
    Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. 
- 
    Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. 
- 
    Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. 
- 
    In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like TBLPROPERTIES (parquet.compression 'NONE'). This happens for ORC Hive table properties likeTBLPROPERTIES (orc.compress 'NONE')in case ofspark.sql.hive.convertMetastoreOrc=true, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example,CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files.
- 
    Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, CREATE TABLE t(id int) STORED AS ORCwould be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark’s ORC data source table and ORC vectorization would be applied. To setfalsetospark.sql.hive.convertMetastoreOrcrestores the previous behavior.
- 
    In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the “id,name” header and one row “1234”. In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set spark.sql.csv.parser.columnPruning.enabledtofalse.
- 
    Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting spark.sql.statistics.parallelFileListingInStatsComputation.enabledtoFalse.
- 
    Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. 
- 
    Since Spark 2.4, empty strings are saved as quoted empty strings "". In version 2.3 and earlier, empty strings are equal tonullvalues and do not reflect to any characters in saved CSV files. For example, the row of"a", null, "", 1was written asa,,,1. Since Spark 2.4, the same row is saved asa,,"",1. To restore the previous behavior, set the CSV optionemptyValueto empty (not quoted) string.
- 
    Since Spark 2.4, The LOAD DATA command supports wildcard ?and*, which match any one character, and zero or more characters, respectively. Example:LOAD DATA INPATH '/tmp/folder*/'orLOAD DATA INPATH '/tmp/part-?'. Special Characters likespacealso now work in paths. Example:LOAD DATA INPATH '/tmp/folder name/'.
- 
    In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, SELECT 1 FROM range(10) HAVING trueis executed asSELECT 1 FROM range(10) WHERE trueand returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which meansSELECT 1 FROM range(10) HAVING truewill return only one row. To restore the previous behavior, setspark.sql.legacy.parser.havingWithoutGroupByAsWheretotrue.
- In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether spark.sql.caseSensitiveis set totrueorfalse. Since 2.4, whenspark.sql.caseSensitiveis set tofalse, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables whenspark.sql.hive.convertMetastoreParquetis set totrue.
Upgrading from Spark SQL 2.2 to 2.3
- 
    Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named _corrupt_recordby default). For example,spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()andspark.read.schema(schema).json(file).select("_corrupt_record").show(). Instead, you can cache or save the parsed results and then send the same query. For example,val df = spark.read.schema(schema).json(file).cache()and thendf.filter($"_corrupt_record".isNotNull).count().
- 
    The percentile_approxfunction previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.
- 
    Since Spark 2.3, the Join/Filter’s deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. 
- Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below:
    InputA \ InputB NullType IntegerType LongType DecimalType(38,0)* DoubleType DateType TimestampType StringType NullType NullType IntegerType LongType DecimalType(38,0) DoubleType DateType TimestampType StringType IntegerType IntegerType IntegerType LongType DecimalType(38,0) DoubleType StringType StringType StringType LongType LongType LongType LongType DecimalType(38,0) StringType StringType StringType StringType DecimalType(38,0)* DecimalType(38,0) DecimalType(38,0) DecimalType(38,0) DecimalType(38,0) StringType StringType StringType StringType DoubleType DoubleType DoubleType StringType StringType DoubleType StringType StringType StringType DateType DateType StringType StringType StringType StringType DateType TimestampType StringType TimestampType TimestampType StringType StringType StringType StringType TimestampType TimestampType StringType StringType StringType StringType StringType StringType StringType StringType StringType StringType Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like BigInteger/BigInt. For example, 1.1 is inferred as double type.
- 
    Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section Join Strategy Hints for SQL Queries and SPARK-22489. 
- 
    Since Spark 2.3, when all inputs are binary, functions.concat()returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, setspark.sql.function.concatBinaryAsStringtotrue.
- 
    Since Spark 2.3, when all inputs are binary, SQL elt()returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, setspark.sql.function.eltOutputAsStringtotrue.
- 
    Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive’s new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - 
        The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, i.e. addition ( +), subtraction (-), multiplication (*), division (/), remainder (%) and positive modulus (pmod).
- 
        Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. 
- 
        The configuration spark.sql.decimalOperations.allowPrecisionLosshas been introduced. It defaults totrue, which means the new behavior described here; if set tofalse, Spark uses previous rules, i.e. it doesn’t adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible.
 
- 
        
- 
    Un-aliased subquery’s semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: SELECT v.i from (SELECT i FROM v), Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See SPARK-20690 and SPARK-21335 for more details.
- When creating a SparkSessionwithSparkSession.builder.getOrCreate(), if there is an existingSparkContext, the builder was trying to update theSparkConfof the existingSparkContextwith configurations specified to the builder, but theSparkContextis shared by allSparkSessions, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating aSparkSession.
Upgrading from Spark SQL 2.1 to 2.2
- 
    Spark 2.1.1 introduced a new configuration key: spark.sql.hive.caseSensitiveInferenceMode. It had a default setting ofNEVER_INFER, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting’s default value toINFER_AND_SAVEto restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With theINFER_AND_SAVEconfiguration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely setspark.sql.hive.caseSensitiveInferenceModetoNEVER_INFERto avoid the initial overhead of schema inference. Note that with the new defaultINFER_AND_SAVEsetting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table’s first access.
- 
    Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). 
- 
    Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using ALTER VIEW ASorCREATE OR REPLACE VIEW ASwith newer Spark versions.
Upgrading from Spark SQL 2.0 to 2.1
- 
    Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as ALTER TABLE PARTITION ... SET LOCATIONare now available for tables created with the Datasource API.- 
        Legacy datasource tables can be migrated to this format via the MSCK REPAIR TABLEcommand. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance.
- 
        To determine if a table has been migrated, look for the PartitionProvider: Catalogattribute when issuingDESCRIBE FORMATTEDon the table.
 
- 
        
- 
    Changes to INSERT OVERWRITE TABLE ... PARTITION ...behavior for Datasource tables.- 
        In prior Spark versions INSERT OVERWRITEoverwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten.
- 
        Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. 
 
- 
        
Upgrading from Spark SQL 1.6 to 2.0
- 
    SparkSessionis now the new entry point of Spark that replaces the oldSQLContextandHiveContext. Note that the old SQLContext and HiveContext are kept for backward compatibility. A newcataloginterface is accessible fromSparkSession- existing API on databases and tables access such aslistTables,createExternalTable,dropTempView,cacheTableare moved here.
- 
    Dataset API and DataFrame API are unified. In Scala, DataFramebecomes a type alias forDataset[Row], while Java API users must replaceDataFramewithDataset<Row>. Both the typed transformations (e.g.,map,filter, andgroupByKey) and untyped transformations (e.g.,selectandgroupBy) are available on the Dataset class. Since compile-time type-safety in Python and R is not a language feature, the concept of Dataset does not apply to these languages’ APIs. Instead,DataFrameremains the primary programming abstraction, which is analogous to the single-node data frame notion in these languages.
- 
    Dataset and DataFrame API unionAllhas been deprecated and replaced byunion
- 
    Dataset and DataFrame API explodehas been deprecated, alternatively, usefunctions.explode()withselectorflatMap
- 
    Dataset and DataFrame API registerTempTablehas been deprecated and replaced bycreateOrReplaceTempView
- 
    Changes to CREATE TABLE ... LOCATIONbehavior for Hive tables.- 
        From Spark 2.0, CREATE TABLE ... LOCATIONis equivalent toCREATE EXTERNAL TABLE ... LOCATIONin order to prevent accidental dropping the existing data in the user-provided locations. That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. Note that this is different from the Hive behavior.
- 
        As a result, DROP TABLEstatements on those tables will not remove the data.
 
- 
        
- 
    spark.sql.parquet.cacheMetadatais no longer used. See SPARK-13664 for details.
Upgrading from Spark SQL 1.5 to 1.6
- From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC
connection owns a copy of their own SQL configuration and temporary function registry. Cached
tables are still shared though. If you prefer to run the Thrift server in the old single-session
mode, please set option spark.sql.hive.thriftServer.singleSessiontotrue. You may either add this option tospark-defaults.conf, or pass it tostart-thriftserver.shvia--conf:
   ./sbin/start-thriftserver.sh \
     --conf spark.sql.hive.thriftServer.singleSession=true \
     ...
   - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType from numeric types. See SPARK-11724 for details.
Upgrading from Spark SQL 1.4 to 1.5
- 
    Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with code generation for expression evaluation. These features can both be disabled by setting spark.sql.tungsten.enabledtofalse.
- 
    Parquet schema merging is no longer enabled by default. It can be re-enabled by setting spark.sql.parquet.mergeSchematotrue.
- 
    In-memory columnar storage partition pruning is on by default. It can be disabled by setting spark.sql.inMemoryColumnarStorage.partitionPruningtofalse.
- 
    Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum precision of 38. When inferring schema from BigDecimalobjects, a precision of (38, 18) is now used. When no precision is specified in DDL then the default remainsDecimal(10, 0).
- 
    Timestamps are now stored at a precision of 1us, rather than 1ns 
- 
    In the sqldialect, floating point numbers are now parsed as decimal. HiveQL parsing remains unchanged.
- 
    The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). 
- 
    JSON data source will not automatically load new files that are created by other applications (i.e. files that are not inserted to the dataset through Spark SQL). For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), users can use REFRESH TABLESQL command orHiveContext’srefreshTablemethod to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate the DataFrame and the new DataFrame will include new files.
Upgrading from Spark SQL 1.3 to 1.4
DataFrame data reader/writer interface
Based on user feedback, we created a new, more fluid API for reading data in (SQLContext.read)
and writing data out (DataFrame.write),
and deprecated the old APIs (e.g., SQLContext.parquetFile, SQLContext.jsonFile).
See the API docs for SQLContext.read (
  Scala,
  Java,
  Python
) and DataFrame.write (
  Scala,
  Java,
  Python
) more information.
DataFrame.groupBy retains grouping columns
Based on user feedback, we changed the default behavior of DataFrame.groupBy().agg() to retain the
grouping columns in the resulting DataFrame. To keep the behavior in 1.3, set spark.sql.retainGroupColumns to false.
// In 1.3.x, in order for the grouping column "department" to show up,
// it must be included explicitly as part of the agg function call.
df.groupBy("department").agg($"department", max("age"), sum("expense"))
// In 1.4+, grouping column "department" is included automatically.
df.groupBy("department").agg(max("age"), sum("expense"))
// Revert to 1.3 behavior (not retaining grouping column) by:
sqlContext.setConf("spark.sql.retainGroupColumns", "false")// In 1.3.x, in order for the grouping column "department" to show up,
// it must be included explicitly as part of the agg function call.
df.groupBy("department").agg(col("department"), max("age"), sum("expense"));
// In 1.4+, grouping column "department" is included automatically.
df.groupBy("department").agg(max("age"), sum("expense"));
// Revert to 1.3 behavior (not retaining grouping column) by:
sqlContext.setConf("spark.sql.retainGroupColumns", "false");import pyspark.sql.functions as func
# In 1.3.x, in order for the grouping column "department" to show up,
# it must be included explicitly as part of the agg function call.
df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense"))
# In 1.4+, grouping column "department" is included automatically.
df.groupBy("department").agg(func.max("age"), func.sum("expense"))
# Revert to 1.3.x behavior (not retaining grouping column) by:
sqlContext.setConf("spark.sql.retainGroupColumns", "false")Behavior change on DataFrame.withColumn
Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added as a new column with its specified name in the result DataFrame even if there may be any existing columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different name from names of all existing columns or replacing existing columns of the same name.
Note that this change is only for Scala API, not for PySpark and SparkR.
Upgrading from Spark SQL 1.0-1.2 to 1.3
In Spark 1.3 we removed the “Alpha” label from Spark SQL and as part of this did a cleanup of the available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked as unstable (i.e., DeveloperAPI or Experimental).
Rename of SchemaRDD to DataFrame
The largest change that users will notice when upgrading to Spark SQL 1.3 is that SchemaRDD has
been renamed to DataFrame. This is primarily because DataFrames no longer inherit from RDD
directly, but instead provide most of the functionality that RDDs provide though their own
implementation. DataFrames can still be converted to RDDs by calling the .rdd method.
In Scala, there is a type alias from SchemaRDD to DataFrame to provide source compatibility for
some use cases. It is still recommended that users update their code to use DataFrame instead.
Java and Python users will need to update their code.
Unification of the Java and Scala APIs
Prior to Spark 1.3 there were separate Java compatible classes (JavaSQLContext and JavaSchemaRDD)
that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users
of either language should use SQLContext and DataFrame. In general these classes try to
use types that are usable from both languages (i.e. Array instead of language-specific collections).
In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading
is used instead.
Additionally, the Java specific types API has been removed. Users of both Scala and Java should
use the classes present in org.apache.spark.sql.types to describe schema programmatically.
Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)
Many of the code examples prior to Spark 1.3 started with import sqlContext._, which brought
all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit
conversions for converting RDDs into DataFrames into an object inside of the SQLContext.
Users should now write import sqlContext.implicits._.
Additionally, the implicit conversions now only augment RDDs that are composed of Products (i.e.,
case classes or tuples) with a method toDF, instead of applying automatically.
When using function inside of the DSL (now replaced with the DataFrame API) users used to import
org.apache.spark.sql.catalyst.dsl. Instead the public dataframe functions API should be used:
import org.apache.spark.sql.functions._.
Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)
Spark 1.3 removes the type aliases that were present in the base sql package for DataType. Users
should instead import the classes in org.apache.spark.sql.types
UDF Registration Moved to sqlContext.udf (Java & Scala)
Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been
moved into the udf object in SQLContext.
sqlContext.udf.register("strLen", (s: String) => s.length())sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType);Python UDF registration is unchanged.
Compatibility with Apache Hive
Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently, Hive SerDes and UDFs are based on built-in Hive, and Spark SQL can be connected to different versions of Hive Metastore (from 0.12.0 to 2.3.9 and 3.0.0 to 3.1.3. Also see Interacting with Different Versions of Hive Metastore).
Deploying in Existing Hive Warehouses
The Spark SQL Thrift JDBC server is designed to be “out of the box” compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables.
Supported Hive Features
Spark SQL supports the vast majority of Hive features, such as:
- Hive query statements, including:
    - SELECT
- GROUP BY
- ORDER BY
- DISTRIBUTE BY
- CLUSTER BY
- SORT BY
 
- All Hive operators, including:
    - Relational operators (=,<=>,==,<>,<,>,>=,<=, etc)
- Arithmetic operators (+,-,*,/,%, etc)
- Logical operators (AND,OR, etc)
- Complex type constructors
- Mathematical functions (sign,ln,cos, etc)
- String functions (instr,length,printf, etc)
 
- Relational operators (
- User defined functions (UDF)
- User defined aggregation functions (UDAF)
- User defined serialization formats (SerDes)
- Window functions
- Joins
    - JOIN
- {LEFT|RIGHT|FULL} OUTER JOIN
- LEFT SEMI JOIN
- LEFT ANTI JOIN
- CROSS JOIN
 
- Unions
- Sub-queries
    - 
        Sub-queries in the FROM Clause SELECT col FROM (SELECT a + b AS col FROM t1) t2
- 
        Sub-queries in WHERE Clause - 
            Correlated or non-correlated IN and NOT IN statement in WHERE Clause SELECT col FROM t1 WHERE col IN (SELECT a FROM t2 WHERE t1.a = t2.a) SELECT col FROM t1 WHERE col IN (SELECT a FROM t2)
- 
            Correlated or non-correlated EXISTS and NOT EXISTS statement in WHERE Clause SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t1.a = t2.a AND t2.a > 10) SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t2.a > 10)
- 
            Non-correlated IN and NOT IN statement in JOIN Condition SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND t1.a IN (SELECT a FROM t3)
- 
            Non-correlated EXISTS and NOT EXISTS statement in JOIN Condition SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)
 
- 
            
 
- 
        
- Sampling
- Explain
- Partitioned tables including dynamic partition insertion
- View
    - 
        If column aliases are not specified in view definition queries, both Spark and Hive will generate alias names, but in different ways. In order for Spark to be able to read views created by Hive, users should explicitly specify column aliases in view definition queries. As an example, Spark cannot read v1created as below by Hive.CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2;Instead, you should create v1as below with column aliases explicitly specified.CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2;
 
- 
        
- All Hive DDL Functions, including:
    - CREATE TABLE
- CREATE TABLE AS SELECT
- CREATE TABLE LIKE
- ALTER TABLE
 
- Most Hive Data types, including:
    - TINYINT
- SMALLINT
- INT
- BIGINT
- BOOLEAN
- FLOAT
- DOUBLE
- STRING
- BINARY
- TIMESTAMP
- DATE
- ARRAY<>
- MAP<>
- STRUCT<>
 
Unsupported Hive Functionality
Below is a list of Hive features that we don’t support yet. Most of these features are rarely used in Hive deployments.
Esoteric Hive Features
- UNIONtype
- Unique join
- Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at the moment and only supports populating the sizeInBytes field of the hive metastore.
Hive Input/Output Formats
- File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat.
- Hadoop archive
Hive Optimizations
A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are less important due to Spark SQL’s in-memory computational model. Others are slotted for future releases of Spark SQL.
- Block-level bitmap indexes and virtual columns (used to build indexes)
- Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you
need to control the degree of parallelism post-shuffle using “SET spark.sql.shuffle.partitions=[num_tasks];”.
- Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still launches tasks to compute the result.
- Skew data flag: Spark SQL does not follow the skew data flags in Hive.
- STREAMTABLEhint in join: Spark SQL does not follow the- STREAMTABLEhint.
- Merge multiple small files for query results: if the result output contains multiple small files, Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that.
Hive UDF/UDTF/UDAF
Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs:
- getRequiredJarsand- getRequiredFiles(- UDFand- GenericUDF) are functions to automatically include additional resources required by this UDF.
- initialize(StructObjectInspector)in- GenericUDTFis not supported yet. Spark SQL currently uses a deprecated interface- initialize(ObjectInspector[])only.
- configure(- GenericUDF,- GenericUDTF, and- GenericUDAFEvaluator) is a function to initialize functions with- MapredContext, which is inapplicable to Spark.
- close(- GenericUDFand- GenericUDAFEvaluator) is a function to release associated resources. Spark SQL does not call this function when tasks finish.
- reset(- GenericUDAFEvaluator) is a function to re-initialize aggregation for reusing the same aggregation. Spark SQL currently does not support the reuse of aggregation.
- getWindowingEvaluator(- GenericUDAFEvaluator) is a function to optimize aggregation by evaluating an aggregate over a fixed window.
Incompatible Hive UDF
Below are the scenarios in which Hive and Spark generate different results:
- SQRT(n)If n < 0, Hive returns null, Spark SQL returns NaN.
- ACOS(n)If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.
- ASIN(n)If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.
- CAST(n AS TIMESTAMP)If n is integral numbers, Hive treats n as milliseconds, Spark SQL treats n as seconds.