Back to home page

OSCL-LXR

 
 

    


0001 ---
0002 layout: global
0003 title: "Migration Guide: SQL, Datasets and DataFrame"
0004 displayTitle: "Migration Guide: SQL, Datasets and DataFrame"
0005 license: |
0006   Licensed to the Apache Software Foundation (ASF) under one or more
0007   contributor license agreements.  See the NOTICE file distributed with
0008   this work for additional information regarding copyright ownership.
0009   The ASF licenses this file to You under the Apache License, Version 2.0
0010   (the "License"); you may not use this file except in compliance with
0011   the License.  You may obtain a copy of the License at
0012 
0013      http://www.apache.org/licenses/LICENSE-2.0
0014 
0015   Unless required by applicable law or agreed to in writing, software
0016   distributed under the License is distributed on an "AS IS" BASIS,
0017   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0018   See the License for the specific language governing permissions and
0019   limitations under the License.
0020 ---
0021 
0022 * Table of contents
0023 {:toc}
0024 
0025 ## Upgrading from Spark SQL 2.4 to 3.0
0026 
0027 ### Dataset/DataFrame APIs
0028 
0029   - In Spark 3.0, the Dataset and DataFrame API `unionAll` is no longer deprecated. It is an alias for `union`.
0030 
0031   - In Spark 2.4 and below, `Dataset.groupByKey` results 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 of `ds.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 configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`.
0032 
0033   - In Spark 3.0, the column metadata will always be propagated in the API `Column.name` and `Column.as`. In Spark version 2.4 and earlier, the metadata of `NamedExpression` is set as the `explicitMetadata` for the new column at the time the API is called, it won't change even if the underlying `NamedExpression` changes metadata. To restore the behavior before Spark 2.4, you can use the API `as(alias: String, metadata: Metadata)` with explicit metadata.
0034 
0035 ### DDL Statements
0036 
0037   - 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 `string` to `int` and `double` to `boolean` are 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 valid `Cast`. 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 option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior.
0038 
0039   - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set.
0040 
0041   - Spark 2.4 and below: the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. In 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`.
0042 
0043   - 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.
0044 
0045   - In Spark 3.0, the properties listing below become reserved; commands fail if you specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. You need their specific clauses to specify them, for example, `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. You can set `spark.sql.legacy.notReserveProperties` to `true` to ignore the `ParseException`, 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'`.
0046 
0047     | Property (case sensitive) | Database Reserved | Table Reserved | Remarks |
0048     | ------------------------- | ----------------- | -------------- | ------- |
0049     | provider                 | no                | yes            | For tables, use the `USING` clause to specify it. Once set, it can't be changed. |
0050     | location                 | yes               | yes            | For databases and tables, use the `LOCATION` clause to specify it. |
0051     | owner                    | yes               | yes            | For databases and tables, it is determined by the user who runs spark and create the table. |
0052 
0053  
0054   - In Spark 3.0, you can use `ADD FILE` to add file directories as well. Earlier you could add only single files using this command. To restore the behavior of earlier versions, set `spark.sql.legacy.addSingleFileInAddFile` to `true`.
0055 
0056   - In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`.
0057 
0058   - In Spark 3.0, `SHOW CREATE TABLE` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE AS SERDE` command instead.
0059 
0060   - 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.
0061 
0062 ### UDFs and Built-in Functions
0063 
0064   - In Spark 3.0, the `date_add` and `date_sub` functions 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')` causes `AnalysisException`. Note that, string literals are still allowed, but Spark will throw `AnalysisException` if 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 of `1964-06-04`.
0065 
0066   - In Spark 3.0, the function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, for example, `percentile_approx(10.0, 0.2, 1.8D)` causes `AnalysisException`. In Spark version 2.4 and below, if `accuracy` is fractional or string value, it is coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`.
0067 
0068   - 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, set `spark.sql.legacy.allowHashOnMapType` to `true`.
0069 
0070   - In Spark 3.0, when the `array`/`map` function is called without any parameters, it returns an empty collection with `NullType` as element type. In Spark version 2.4 and below, it returns an empty collection with `StringType` as element type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createEmptyCollectionUsingStringType` to `true`.
0071 
0072   - In Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`.
0073 
0074   - 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 use `map_entries` function 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.
0075 
0076   - 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.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. In Spark 3.0, Spark throws `RuntimeException` when duplicated keys are found. You can set `spark.sql.mapKeyDedupPolicy` to `LAST_WIN` to 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.
0077 
0078   - 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 set `spark.sql.legacy.allowUntypedScalaUDF` to true to keep using it. In Spark version 2.4 and below, if `org.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 column `x` is 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.
0079 
0080   - In Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, that is, if the `predicate` returns any `null`s and no `true` is obtained, then `exists` returns `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` is `null`. The previous behaviorcan be restored by setting `spark.sql.legacy.followThreeValuedLogicInArrayExists` to `false`.
0081 
0082   - In Spark 3.0, the `add_months` function 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)` results `2019-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 to `2019-02-28` results in `2019-03-31`.
0083 
0084   - In Spark version 2.4 and below, the `current_timestamp` function 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.
0085 
0086   - 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.
0087 
0088   - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`.
0089 
0090   - In Spark 3.0, the `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in a case-insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behavior change is illustrated in the table below:
0091 
0092     | Operation | Result before Spark 3.0 | Result in Spark 3.0 |
0093     | --------- | ----------------------- | ------------------- |
0094     | CAST('infinity' AS DOUBLE) | NULL | Double.PositiveInfinity |
0095     | CAST('+infinity' AS DOUBLE) | NULL | Double.PositiveInfinity |
0096     | CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity |
0097     | CAST('inf' AS DOUBLE) | NULL | Double.PositiveInfinity |
0098     | CAST('-infinity' AS DOUBLE) | NULL | Double.NegativeInfinity |
0099     | CAST('-inf' AS DOUBLE) | NULL | Double.NegativeInfinity |
0100     | CAST('infinity' AS FLOAT) | NULL | Float.PositiveInfinity |
0101     | CAST('+infinity' AS FLOAT) | NULL | Float.PositiveInfinity |
0102     | CAST('inf' AS FLOAT) | NULL | Float.PositiveInfinity |
0103     | CAST('+inf' AS FLOAT) | NULL | Float.PositiveInfinity |
0104     | CAST('-infinity' AS FLOAT) | NULL | Float.NegativeInfinity |
0105     | CAST('-inf' AS FLOAT) | NULL | Float.NegativeInfinity |
0106     | CAST('nan' AS DOUBLE) | NULL | Double.Nan |
0107     | CAST('nan' AS FLOAT) | NULL | Float.NaN |
0108 
0109   - 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 like `interval 1 days 2 hours`.
0110 
0111   - 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)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-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 is `null`, while to datetimes, only the trailing spaces (= ASCII 32) are removed.
0112 
0113 ### Query Engine
0114 
0115   - In Spark version 2.4 and below, SQL queries such as `FROM <table>` or `FROM <table> UNION ALL FROM <table>` are supported by accident. In hive-style `FROM <table> SELECT <expr>`, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. These queries are treated as invalid in Spark 3.0.
0116 
0117   - 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.
0118 
0119   - 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 set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`.
0120 
0121   - In Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`.
0122 
0123   - In Spark 3.0, negative scale of decimal is not allowed by default, for example, data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and below, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal` to `true`.
0124 
0125   - 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'` returns `1.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 string `1`.
0126 
0127   - 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(...);`, then `df1.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, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`.
0128 
0129   - In Spark 3.0, `spark.sql.legacy.ctePrecedencePolicy` is introduced to control the behavior for name conflicting in the nested WITH clause. By default value `EXCEPTION`, Spark throws an AnalysisException, it forces users to choose the specific substitution order they wanted. If set to `CORRECTED` (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `LEGACY`, the result is `1` which is the behavior in version 2.4 and below.
0130 
0131   - In Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join.
0132 
0133   - 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.
0134 
0135   - 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`.
0136 
0137   - 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.time` packages that are based on [ISO chronology](https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and below, those operations are performed using the hybrid calendar ([Julian + Gregorian](https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API:
0138 
0139     * 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_timestamp` functions when patterns specified by users is used for parsing and formatting. In Spark 3.0, we define our own pattern strings in `sql-ref-datetime-pattern.md`, which is implemented via `java.time.format.DateTimeFormatter` under the hood. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and below, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in [simpleDateFormat](https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html). The old behavior can be restored by setting `spark.sql.legacy.timeParserPolicy` to `LEGACY`.
0140 
0141     * The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions 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.
0142 
0143     * The JDBC options `lowerBound` and `upperBound` are 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 config `spark.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.
0144 
0145     * Formatting `TIMESTAMP` and `DATE` literals.
0146 
0147     * Creating typed `TIMESTAMP` and `DATE` literals from strings. In Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('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 config `spark.sql.session.timeZone` is 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 typed `TIMESTAMP` and `DATE` literals.
0148 
0149   - In Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and below, the conversion uses the default time zone of the Java virtual machine.
0150 
0151   - In Spark 3.0, Spark casts `String` to `Date/Timestamp` in binary comparisons with dates/timestamps. The previous behavior of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString.enabled` to `true`.
0152 
0153   - 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:
0154     * `epoch [zoneId]` - 1970-01-01
0155     * `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`
0156     * `yesterday [zoneId]` - the current date - 1
0157     * `tomorrow [zoneId]` - the current date + 1
0158     * `now` - the date of running the current query. It has the same notion as today
0159 
0160     For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values:
0161     * `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero)
0162     * `today [zoneId]` - midnight today
0163     * `yesterday [zoneId]` - midnight yesterday
0164     * `tomorrow [zoneId]` - midnight tomorrow
0165     * `now` - current query start time
0166 
0167     For example `SELECT timestamp 'tomorrow';`.
0168 
0169   - Since Spark 3.0, when using `EXTRACT` expression to extract the second field from date/timestamp values, the result will be a `DecimalType(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'))` results `10.100000`.  In Spark version 2.4 and earlier, it returns an `IntegerType` value and the result for the former example is `10`.
0170 
0171 ### Data Sources
0172 
0173   - 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.caseSensitiveInferenceMode` to `INFER_AND_SAVE`.
0174 
0175   - 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.validatePartitionColumns` to `false`.
0176 
0177   - 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.ignoreMissingFiles` is `true` (default `false`). 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 during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time.
0178 
0179   - 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`. For `FloatType`, `DoubleType`, `DateType` and `TimestampType`, it fails on empty strings and throws exceptions. Spark 3.0 disallows empty strings and will throw an exception for data types except for `StringType` and `BinaryType`. The previous behavior of allowing an empty string can be restored by setting `spark.sql.legacy.json.allowEmptyString.enabled` to `true`.
0180 
0181   - In Spark version 2.4 and below, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. In Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully.
0182 
0183   - In Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inference.
0184 
0185   - In Spark version 2.4 and below, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. In Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully.
0186 
0187   - 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.
0188 
0189   - 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.
0190 
0191 ### Others
0192 
0193   - In Spark version 2.4, when a Spark session is created via `cloneSession()`, the newly created Spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent Spark session. In Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. You can restore the old behavior by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`.
0194 
0195   - In Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it falls back to the `hive-site.xml` file present in the `Hadoop configuration` of `SparkContext`.
0196 
0197   - In Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example:
0198 
0199     | Query | Spark 2.4 | Spark 3.0 |
0200     | ----- | --------- | --------- |
0201     |`SELECT CAST(1 AS decimal(38, 18));` | 1 | 1.000000000000000000 |
0202 
0203   - In Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3 and it brings following impacts:
0204 
0205     * You may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore you want to connect to. For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1.
0206 
0207     * You need to migrate your custom SerDes to Hive 2.3 or build your own Spark with `hive-1.2` profile. See [HIVE-15167](https://issues.apache.org/jira/browse/HIVE-15167) for more details.
0208 
0209     * The decimal string representation can be different between Hive 1.2 and Hive 2.3 when using `TRANSFORM` operator 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.
0210 
0211 ## Upgrading from Spark SQL 2.4.5 to 2.4.6
0212 
0213   - In Spark 2.4.6, the `RESET` command does not reset the static SQL configuration values to the default. It only clears the runtime SQL configuration values.
0214 
0215 ## Upgrading from Spark SQL 2.4.4 to 2.4.5
0216 
0217   - Since Spark 2.4.5, `TRUNCATE TABLE` command tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, set `spark.sql.truncateTable.ignorePermissionAcl.enabled` to `true`.
0218 
0219   - Since Spark 2.4.5, `spark.sql.legacy.mssqlserver.numericMapping.enabled` configuration 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, set `spark.sql.legacy.mssqlserver.numericMapping.enabled` to `true`.
0220 
0221 ## Upgrading from Spark SQL 2.4.3 to 2.4.4
0222 
0223   - Since Spark 2.4.4, according to [MsSqlServer Guide](https://docs.microsoft.com/en-us/sql/connect/jdbc/using-basic-data-types?view=sql-server-2017), MsSQLServer JDBC Dialect uses ShortType and FloatType for SMALLINT and REAL, respectively. Previously, IntegerType and DoubleType is used.
0224 
0225 ## Upgrading from Spark SQL 2.4 to 2.4.1
0226 
0227   - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was
0228     inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code.
0229     Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30"
0230     need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise,
0231     the extremely short interval that results will likely cause applications to fail.
0232 
0233   - 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, but `Seq("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.
0234 
0235 ## Upgrading from Spark SQL 2.3 to 2.4
0236 
0237   - 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_contains` function 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.
0238     <table class="table">
0239         <tr>
0240           <th>
0241             <b>Query</b>
0242           </th>
0243           <th>
0244             <b>Spark 2.3 or Prior</b>
0245           </th>
0246           <th>
0247             <b>Spark 2.4</b>
0248           </th>
0249           <th>
0250             <b>Remarks</b>
0251           </th>
0252         </tr>
0253         <tr>
0254           <td>
0255             <code>SELECT array_contains(array(1), 1.34D);</code>
0256           </td>
0257           <td>
0258             <code>true</code>
0259           </td>
0260           <td>
0261             <code>false</code>
0262           </td>
0263           <td>
0264             In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively.
0265           </td>
0266         </tr>
0267         <tr>
0268           <td>
0269             <code>SELECT array_contains(array(1), '1');</code>
0270           </td>
0271           <td>
0272             <code>true</code>
0273           </td>
0274           <td>
0275             <code>AnalysisException</code> is thrown.
0276           </td>
0277           <td>
0278             Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, <code>AnalysisException</code> is thrown since integer type can not be promoted to string type in a loss-less manner.
0279           </td>
0280         </tr>
0281         <tr>
0282           <td>
0283             <code>SELECT array_contains(array(1), 'anystring');</code>
0284           </td>
0285           <td>
0286             <code>null</code>
0287           </td>
0288           <td>
0289             <code>AnalysisException</code> is thrown.
0290           </td>
0291           <td>
0292             Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, <code>AnalysisException</code> is thrown since integer type can not be promoted to string type in a loss-less manner.
0293           </td>
0294         </tr>
0295     </table>
0296 
0297   - 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. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite.
0298 
0299   - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions 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.
0300 
0301   - 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.enabled` with a default value of `false`. When this property is set to `true`, 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.
0302 
0303   - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970.
0304 
0305   - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older.
0306 
0307   - 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.
0308 
0309   - 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)` but ``UDF:f(col0 AS `colA`)``.
0310 
0311   - 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.
0312 
0313   - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0.
0314 
0315   - 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 `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0.
0316 
0317   - 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.
0318 
0319   - 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.
0320 
0321   - 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.
0322 
0323   - 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 like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.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.
0324 
0325   - 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 ORC` would 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 set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior.
0326 
0327   - 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.enabled` to `false`.
0328 
0329   - 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.enabled` to `False`.
0330 
0331   - 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.
0332 
0333   - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string.
0334 
0335   - 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*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`.
0336 
0337   - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true`  and 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 means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`.
0338 
0339   - 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.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, 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 when `spark.sql.hive.convertMetastoreParquet` is set to `true`.
0340 
0341 ## Upgrading from Spark SQL 2.2 to 2.3
0342 
0343   - 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_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.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 then `df.filter($"_corrupt_record".isNotNull).count()`.
0344 
0345   - The `percentile_approx` function 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.
0346 
0347   - 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.
0348 
0349   - 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:
0350     <table class="table">
0351       <tr>
0352         <th>
0353           <b>InputA \ InputB</b>
0354         </th>
0355         <th>
0356           <b>NullType</b>
0357         </th>
0358         <th>
0359           <b>IntegerType</b>
0360         </th>
0361         <th>
0362           <b>LongType</b>
0363         </th>
0364         <th>
0365           <b>DecimalType(38,0)*</b>
0366         </th>
0367         <th>
0368           <b>DoubleType</b>
0369         </th>
0370         <th>
0371           <b>DateType</b>
0372         </th>
0373         <th>
0374           <b>TimestampType</b>
0375         </th>
0376         <th>
0377           <b>StringType</b>
0378         </th>
0379       </tr>
0380       <tr>
0381         <td>
0382           <b>NullType</b>
0383         </td>
0384         <td>NullType</td>
0385         <td>IntegerType</td>
0386         <td>LongType</td>
0387         <td>DecimalType(38,0)</td>
0388         <td>DoubleType</td>
0389         <td>DateType</td>
0390         <td>TimestampType</td>
0391         <td>StringType</td>
0392       </tr>
0393       <tr>
0394         <td>
0395           <b>IntegerType</b>
0396         </td>
0397         <td>IntegerType</td>
0398         <td>IntegerType</td>
0399         <td>LongType</td>
0400         <td>DecimalType(38,0)</td>
0401         <td>DoubleType</td>
0402         <td>StringType</td>
0403         <td>StringType</td>
0404         <td>StringType</td>
0405       </tr>
0406       <tr>
0407         <td>
0408           <b>LongType</b>
0409         </td>
0410         <td>LongType</td>
0411         <td>LongType</td>
0412         <td>LongType</td>
0413         <td>DecimalType(38,0)</td>
0414         <td>StringType</td>
0415         <td>StringType</td>
0416         <td>StringType</td>
0417         <td>StringType</td>
0418       </tr>
0419       <tr>
0420         <td>
0421           <b>DecimalType(38,0)*</b>
0422         </td>
0423         <td>DecimalType(38,0)</td>
0424         <td>DecimalType(38,0)</td>
0425         <td>DecimalType(38,0)</td>
0426         <td>DecimalType(38,0)</td>
0427         <td>StringType</td>
0428         <td>StringType</td>
0429         <td>StringType</td>
0430         <td>StringType</td>
0431       </tr>
0432       <tr>
0433         <td>
0434           <b>DoubleType</b>
0435         </td>
0436         <td>DoubleType</td>
0437         <td>DoubleType</td>
0438         <td>StringType</td>
0439         <td>StringType</td>
0440         <td>DoubleType</td>
0441         <td>StringType</td>
0442         <td>StringType</td>
0443         <td>StringType</td>
0444       </tr>
0445       <tr>
0446         <td>
0447           <b>DateType</b>
0448         </td>
0449         <td>DateType</td>
0450         <td>StringType</td>
0451         <td>StringType</td>
0452         <td>StringType</td>
0453         <td>StringType</td>
0454         <td>DateType</td>
0455         <td>TimestampType</td>
0456         <td>StringType</td>
0457       </tr>
0458       <tr>
0459         <td>
0460           <b>TimestampType</b>
0461         </td>
0462         <td>TimestampType</td>
0463         <td>StringType</td>
0464         <td>StringType</td>
0465         <td>StringType</td>
0466         <td>StringType</td>
0467         <td>TimestampType</td>
0468         <td>TimestampType</td>
0469         <td>StringType</td>
0470       </tr>
0471       <tr>
0472         <td>
0473           <b>StringType</b>
0474         </td>
0475         <td>StringType</td>
0476         <td>StringType</td>
0477         <td>StringType</td>
0478         <td>StringType</td>
0479         <td>StringType</td>
0480         <td>StringType</td>
0481         <td>StringType</td>
0482         <td>StringType</td>
0483       </tr>
0484     </table>
0485 
0486     Note that, for <b>DecimalType(38,0)*</b>, 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.
0487 
0488   - 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](sql-performance-tuning.html#join-strategy-hints-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489).
0489 
0490   - 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, set `spark.sql.function.concatBinaryAsString` to `true`.
0491 
0492   - 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, set `spark.sql.function.eltOutputAsString` to `true`.
0493 
0494  - 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
0495 
0496     - 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, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`).
0497 
0498     - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them.
0499 
0500     - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. 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.
0501 
0502   - 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](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details.
0503 
0504   - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, 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 a `SparkSession`.
0505 
0506 ## Upgrading from Spark SQL 2.1 to 2.2
0507 
0508   - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration 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 set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, 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.
0509 
0510   - 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).
0511 
0512   - 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 AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions.
0513 
0514 ## Upgrading from Spark SQL 2.0 to 2.1
0515 
0516  - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API.
0517 
0518     - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance.
0519 
0520     - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table.
0521  - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables.
0522 
0523     - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten.
0524 
0525     - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data.
0526 
0527 ## Upgrading from Spark SQL 1.6 to 2.0
0528 
0529  - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and
0530 
0531    `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here.
0532 
0533  - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for
0534    `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset<Row>`. Both the typed
0535    transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g.,
0536    `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in
0537    Python and R is not a language feature, the concept of Dataset does not apply to these languages’
0538    APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the
0539    single-node data frame notion in these languages.
0540 
0541  - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union`
0542 
0543  - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap`
0544 
0545  - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView`
0546 
0547  - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables.
0548 
0549     - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION`
0550       in order to prevent accidental dropping the existing data in the user-provided locations.
0551       That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table.
0552       Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables.
0553       Note that this is different from the Hive behavior.
0554 
0555     - As a result, `DROP TABLE` statements on those tables will not remove the data.
0556 
0557  - `spark.sql.parquet.cacheMetadata` is no longer used.
0558    See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details.
0559 
0560 ## Upgrading from Spark SQL 1.5 to 1.6
0561 
0562  - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC
0563    connection owns a copy of their own SQL configuration and temporary function registry. Cached
0564    tables are still shared though. If you prefer to run the Thrift server in the old single-session
0565    mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add
0566    this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`:
0567 
0568    {% highlight bash %}
0569    ./sbin/start-thriftserver.sh \
0570      --conf spark.sql.hive.thriftServer.singleSession=true \
0571      ...
0572    {% endhighlight %}
0573 
0574  - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This
0575    change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType
0576    from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for
0577    details.
0578 
0579 ## Upgrading from Spark SQL 1.4 to 1.5
0580 
0581  - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with
0582    code generation for expression evaluation. These features can both be disabled by setting
0583    `spark.sql.tungsten.enabled` to `false`.
0584 
0585  - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting
0586    `spark.sql.parquet.mergeSchema` to `true`.
0587 
0588  - In-memory columnar storage partition pruning is on by default. It can be disabled by setting
0589    `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`.
0590 
0591  - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum
0592    precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now
0593    used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`.
0594 
0595  - Timestamps are now stored at a precision of 1us, rather than 1ns
0596 
0597  - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains
0598    unchanged.
0599 
0600  - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM).
0601 
0602  - JSON data source will not automatically load new files that are created by other applications
0603    (i.e. files that are not inserted to the dataset through Spark SQL).
0604    For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore),
0605    users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method
0606    to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate
0607    the DataFrame and the new DataFrame will include new files.
0608 
0609 ## Upgrading from Spark SQL 1.3 to 1.4
0610 
0611 #### DataFrame data reader/writer interface
0612 {:.no_toc}
0613 
0614 Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`)
0615 and writing data out (`DataFrame.write`),
0616 and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`).
0617 
0618 See the API docs for `SQLContext.read` (
0619   <a href="api/scala/org/apache/spark/sql/SQLContext.html#read:DataFrameReader">Scala</a>,
0620   <a href="api/java/org/apache/spark/sql/SQLContext.html#read()">Java</a>,
0621   <a href="api/python/pyspark.sql.html#pyspark.sql.SQLContext.read">Python</a>
0622 ) and `DataFrame.write` (
0623   <a href="api/scala/org/apache/spark/sql/DataFrame.html#write:DataFrameWriter">Scala</a>,
0624   <a href="api/java/org/apache/spark/sql/Dataset.html#write()">Java</a>,
0625   <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame.write">Python</a>
0626 ) more information.
0627 
0628 
0629 #### DataFrame.groupBy retains grouping columns
0630 {:.no_toc}
0631 
0632 Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the
0633 grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`.
0634 
0635 <div class="codetabs">
0636 <div data-lang="scala"  markdown="1">
0637 {% highlight scala %}
0638 
0639 // In 1.3.x, in order for the grouping column "department" to show up,
0640 // it must be included explicitly as part of the agg function call.
0641 df.groupBy("department").agg($"department", max("age"), sum("expense"))
0642 
0643 // In 1.4+, grouping column "department" is included automatically.
0644 df.groupBy("department").agg(max("age"), sum("expense"))
0645 
0646 // Revert to 1.3 behavior (not retaining grouping column) by:
0647 sqlContext.setConf("spark.sql.retainGroupColumns", "false")
0648 
0649 {% endhighlight %}
0650 </div>
0651 
0652 <div data-lang="java"  markdown="1">
0653 {% highlight java %}
0654 
0655 // In 1.3.x, in order for the grouping column "department" to show up,
0656 // it must be included explicitly as part of the agg function call.
0657 df.groupBy("department").agg(col("department"), max("age"), sum("expense"));
0658 
0659 // In 1.4+, grouping column "department" is included automatically.
0660 df.groupBy("department").agg(max("age"), sum("expense"));
0661 
0662 // Revert to 1.3 behavior (not retaining grouping column) by:
0663 sqlContext.setConf("spark.sql.retainGroupColumns", "false");
0664 
0665 {% endhighlight %}
0666 </div>
0667 
0668 <div data-lang="python"  markdown="1">
0669 {% highlight python %}
0670 
0671 import pyspark.sql.functions as func
0672 
0673 # In 1.3.x, in order for the grouping column "department" to show up,
0674 # it must be included explicitly as part of the agg function call.
0675 df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense"))
0676 
0677 # In 1.4+, grouping column "department" is included automatically.
0678 df.groupBy("department").agg(func.max("age"), func.sum("expense"))
0679 
0680 # Revert to 1.3.x behavior (not retaining grouping column) by:
0681 sqlContext.setConf("spark.sql.retainGroupColumns", "false")
0682 
0683 {% endhighlight %}
0684 </div>
0685 
0686 </div>
0687 
0688 
0689 #### Behavior change on DataFrame.withColumn
0690 {:.no_toc}
0691 
0692 Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added
0693 as a new column with its specified name in the result DataFrame even if there may be any existing
0694 columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different
0695 name from names of all existing columns or replacing existing columns of the same name.
0696 
0697 Note that this change is only for Scala API, not for PySpark and SparkR.
0698 
0699 
0700 ## Upgrading from Spark SQL 1.0-1.2 to 1.3
0701 
0702 In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the
0703 available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other
0704 releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked
0705 as unstable (i.e., DeveloperAPI or Experimental).
0706 
0707 #### Rename of SchemaRDD to DataFrame
0708 {:.no_toc}
0709 
0710 The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has
0711 been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD
0712 directly, but instead provide most of the functionality that RDDs provide though their own
0713 implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method.
0714 
0715 In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for
0716 some use cases. It is still recommended that users update their code to use `DataFrame` instead.
0717 Java and Python users will need to update their code.
0718 
0719 #### Unification of the Java and Scala APIs
0720 {:.no_toc}
0721 
0722 Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`)
0723 that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users
0724 of either language should use `SQLContext` and `DataFrame`. In general these classes try to
0725 use types that are usable from both languages (i.e. `Array` instead of language-specific collections).
0726 In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading
0727 is used instead.
0728 
0729 Additionally, the Java specific types API has been removed. Users of both Scala and Java should
0730 use the classes present in `org.apache.spark.sql.types` to describe schema programmatically.
0731 
0732 
0733 #### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)
0734 {:.no_toc}
0735 
0736 Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought
0737 all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit
0738 conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`.
0739 Users should now write `import sqlContext.implicits._`.
0740 
0741 Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e.,
0742 case classes or tuples) with a method `toDF`, instead of applying automatically.
0743 
0744 When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import
0745 `org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used:
0746 `import org.apache.spark.sql.functions._`.
0747 
0748 #### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)
0749 {:.no_toc}
0750 
0751 Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users
0752 should instead import the classes in `org.apache.spark.sql.types`
0753 
0754 #### UDF Registration Moved to `sqlContext.udf` (Java & Scala)
0755 {:.no_toc}
0756 
0757 Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been
0758 moved into the udf object in `SQLContext`.
0759 
0760 <div class="codetabs">
0761 <div data-lang="scala"  markdown="1">
0762 {% highlight scala %}
0763 
0764 sqlContext.udf.register("strLen", (s: String) => s.length())
0765 
0766 {% endhighlight %}
0767 </div>
0768 
0769 <div data-lang="java"  markdown="1">
0770 {% highlight java %}
0771 
0772 sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType);
0773 
0774 {% endhighlight %}
0775 </div>
0776 
0777 </div>
0778 
0779 Python UDF registration is unchanged.
0780 
0781 
0782 
0783 ## Compatibility with Apache Hive
0784 
0785 Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs.
0786 Currently, Hive SerDes and UDFs are based on built-in Hive,
0787 and Spark SQL can be connected to different versions of Hive Metastore
0788 (from 0.12.0 to 2.3.7 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)).
0789 
0790 #### Deploying in Existing Hive Warehouses
0791 {:.no_toc}
0792 
0793 The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive
0794 installations. You do not need to modify your existing Hive Metastore or change the data placement
0795 or partitioning of your tables.
0796 
0797 ### Supported Hive Features
0798 {:.no_toc}
0799 
0800 Spark SQL supports the vast majority of Hive features, such as:
0801 
0802 * Hive query statements, including:
0803   * `SELECT`
0804   * `GROUP BY`
0805   * `ORDER BY`
0806   * `DISTRIBUTE BY`
0807   * `CLUSTER BY`
0808   * `SORT BY`
0809 * All Hive operators, including:
0810   * Relational operators (`=`, `<=>`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc)
0811   * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc)
0812   * Logical operators (`AND`, `&&`, `OR`, `||`, etc)
0813   * Complex type constructors
0814   * Mathematical functions (`sign`, `ln`, `cos`, etc)
0815   * String functions (`instr`, `length`, `printf`, etc)
0816 * User defined functions (UDF)
0817 * User defined aggregation functions (UDAF)
0818 * User defined serialization formats (SerDes)
0819 * Window functions
0820 * Joins
0821   * `JOIN`
0822   * `{LEFT|RIGHT|FULL} OUTER JOIN`
0823   * `LEFT SEMI JOIN`
0824   * `LEFT ANTI JOIN`
0825   * `CROSS JOIN`
0826 * Unions
0827 * Sub-queries
0828   * Sub-queries in the FROM Clause
0829 
0830     ```SELECT col FROM (SELECT a + b AS col FROM t1) t2```
0831   * Sub-queries in WHERE Clause
0832     * Correlated or non-correlated IN and NOT IN statement in WHERE Clause
0833 
0834       ```
0835       SELECT col FROM t1 WHERE col IN (SELECT a FROM t2 WHERE t1.a = t2.a)
0836       SELECT col FROM t1 WHERE col IN (SELECT a FROM t2)
0837       ```
0838     * Correlated or non-correlated EXISTS and NOT EXISTS statement in WHERE Clause
0839 
0840       ```
0841       SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t1.a = t2.a AND t2.a > 10)
0842       SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t2.a > 10)
0843       ```
0844     * Non-correlated IN and NOT IN statement in JOIN Condition
0845 
0846       ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND t1.a IN (SELECT a FROM t3)```
0847 
0848     * Non-correlated EXISTS and NOT EXISTS statement in JOIN Condition
0849 
0850       ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)```
0851 
0852 * Sampling
0853 * Explain
0854 * Partitioned tables including dynamic partition insertion
0855 * View
0856   * If column aliases are not specified in view definition queries, both Spark and Hive will
0857     generate alias names, but in different ways. In order for Spark to be able to read views created
0858     by Hive, users should explicitly specify column aliases in view definition queries. As an
0859     example, Spark cannot read `v1` created as below by Hive.
0860 
0861     ```
0862     CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2;
0863     ```
0864 
0865     Instead, you should create `v1` as below with column aliases explicitly specified.
0866 
0867     ```
0868     CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2;
0869     ```
0870 
0871 * All Hive DDL Functions, including:
0872   * `CREATE TABLE`
0873   * `CREATE TABLE AS SELECT`
0874   * `CREATE TABLE LIKE`
0875   * `ALTER TABLE`
0876 * Most Hive Data types, including:
0877   * `TINYINT`
0878   * `SMALLINT`
0879   * `INT`
0880   * `BIGINT`
0881   * `BOOLEAN`
0882   * `FLOAT`
0883   * `DOUBLE`
0884   * `STRING`
0885   * `BINARY`
0886   * `TIMESTAMP`
0887   * `DATE`
0888   * `ARRAY<>`
0889   * `MAP<>`
0890   * `STRUCT<>`
0891 
0892 ### Unsupported Hive Functionality
0893 {:.no_toc}
0894 
0895 Below is a list of Hive features that we don't support yet. Most of these features are rarely used
0896 in Hive deployments.
0897 
0898 **Major Hive Features**
0899 
0900 * Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL
0901   doesn't support buckets yet.
0902 
0903 
0904 **Esoteric Hive Features**
0905 
0906 * `UNION` type
0907 * Unique join
0908 * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at
0909   the moment and only supports populating the sizeInBytes field of the hive metastore.
0910 
0911 **Hive Input/Output Formats**
0912 
0913 * File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat.
0914 * Hadoop archive
0915 
0916 **Hive Optimizations**
0917 
0918 A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are
0919 less important due to Spark SQL's in-memory computational model. Others are slotted for future
0920 releases of Spark SQL.
0921 
0922 * Block-level bitmap indexes and virtual columns (used to build indexes)
0923 * Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you
0924   need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`".
0925 * Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still
0926   launches tasks to compute the result.
0927 * Skew data flag: Spark SQL does not follow the skew data flags in Hive.
0928 * `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint.
0929 * Merge multiple small files for query results: if the result output contains multiple small files,
0930   Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS
0931   metadata. Spark SQL does not support that.
0932 
0933 **Hive UDF/UDTF/UDAF**
0934 
0935 Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs:
0936 
0937 * `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically
0938   include additional resources required by this UDF.
0939 * `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses
0940   a deprecated interface `initialize(ObjectInspector[])` only.
0941 * `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize
0942   functions with `MapredContext`, which is inapplicable to Spark.
0943 * `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources.
0944   Spark SQL does not call this function when tasks finish.
0945 * `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation.
0946   Spark SQL currently does not support the reuse of aggregation.
0947 * `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating
0948   an aggregate over a fixed window.
0949 
0950 ### Incompatible Hive UDF
0951 {:.no_toc}
0952 
0953 Below are the scenarios in which Hive and Spark generate different results:
0954 
0955 * `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN.
0956 * `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.
0957 * `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.
0958 * `CAST(n AS TIMESTAMP)` If n is integral numbers, Hive treats n as milliseconds, Spark SQL treats n as seconds.