| |
| <!DOCTYPE html> |
| <!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]--> |
| <!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]--> |
| <!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| <meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1"> |
| <title>Spark SQL Upgrading Guide - Spark 2.4.5 Documentation</title> |
| |
| |
| |
| |
| <link rel="stylesheet" href="css/bootstrap.min.css"> |
| <style> |
| body { |
| padding-top: 60px; |
| padding-bottom: 40px; |
| } |
| </style> |
| <meta name="viewport" content="width=device-width"> |
| <link rel="stylesheet" href="css/bootstrap-responsive.min.css"> |
| <link rel="stylesheet" href="css/main.css"> |
| |
| <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script> |
| |
| <link rel="stylesheet" href="css/pygments-default.css"> |
| |
| |
| <!-- Google analytics script --> |
| <script type="text/javascript"> |
| var _gaq = _gaq || []; |
| _gaq.push(['_setAccount', 'UA-32518208-2']); |
| _gaq.push(['_trackPageview']); |
| |
| (function() { |
| var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true; |
| ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js'; |
| var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s); |
| })(); |
| </script> |
| |
| |
| </head> |
| <body> |
| <!--[if lt IE 7]> |
| <p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p> |
| <![endif]--> |
| |
| <!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html --> |
| |
| <div class="navbar navbar-fixed-top" id="topbar"> |
| <div class="navbar-inner"> |
| <div class="container"> |
| <div class="brand"><a href="index.html"> |
| <img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">2.4.5</span> |
| </div> |
| <ul class="nav"> |
| <!--TODO(andyk): Add class="active" attribute to li some how.--> |
| <li><a href="index.html">Overview</a></li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Programming Guides<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="quick-start.html">Quick Start</a></li> |
| <li><a href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a></li> |
| <li><a href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a></li> |
| <li><a href="structured-streaming-programming-guide.html">Structured Streaming</a></li> |
| <li><a href="streaming-programming-guide.html">Spark Streaming (DStreams)</a></li> |
| <li><a href="ml-guide.html">MLlib (Machine Learning)</a></li> |
| <li><a href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li> |
| <li><a href="sparkr.html">SparkR (R on Spark)</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="api/scala/index.html#org.apache.spark.package">Scala</a></li> |
| <li><a href="api/java/index.html">Java</a></li> |
| <li><a href="api/python/index.html">Python</a></li> |
| <li><a href="api/R/index.html">R</a></li> |
| <li><a href="api/sql/index.html">SQL, Built-in Functions</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="cluster-overview.html">Overview</a></li> |
| <li><a href="submitting-applications.html">Submitting Applications</a></li> |
| <li class="divider"></li> |
| <li><a href="spark-standalone.html">Spark Standalone</a></li> |
| <li><a href="running-on-mesos.html">Mesos</a></li> |
| <li><a href="running-on-yarn.html">YARN</a></li> |
| <li><a href="running-on-kubernetes.html">Kubernetes</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="configuration.html">Configuration</a></li> |
| <li><a href="monitoring.html">Monitoring</a></li> |
| <li><a href="tuning.html">Tuning Guide</a></li> |
| <li><a href="job-scheduling.html">Job Scheduling</a></li> |
| <li><a href="security.html">Security</a></li> |
| <li><a href="hardware-provisioning.html">Hardware Provisioning</a></li> |
| <li class="divider"></li> |
| <li><a href="building-spark.html">Building Spark</a></li> |
| <li><a href="https://spark.apache.org/contributing.html">Contributing to Spark</a></li> |
| <li><a href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a></li> |
| </ul> |
| </li> |
| </ul> |
| <!--<p class="navbar-text pull-right"><span class="version-text">v2.4.5</span></p>--> |
| </div> |
| </div> |
| </div> |
| |
| <div class="container-wrapper"> |
| |
| |
| |
| <div class="left-menu-wrapper"> |
| <div class="left-menu"> |
| <h3><a href="sql-programming-guide.html">Spark SQL Guide</a></h3> |
| |
| <ul> |
| |
| <li> |
| <a href="sql-getting-started.html"> |
| |
| Getting Started |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-data-sources.html"> |
| |
| Data Sources |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-performance-tuning.html"> |
| |
| Performance Tuning |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-distributed-sql-engine.html"> |
| |
| Distributed SQL Engine |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-pyspark-pandas-with-arrow.html"> |
| |
| PySpark Usage Guide for Pandas with Apache Arrow |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-migration-guide.html"> |
| |
| Migration Guide |
| |
| </a> |
| </li> |
| |
| |
| |
| <ul> |
| |
| <li> |
| <a href="sql-migration-guide-upgrade.html"> |
| |
| <b>Spark SQL Upgrading Guide</b> |
| |
| </a> |
| </li> |
| |
| |
| |
| <li> |
| <a href="sql-migration-guide-hive-compatibility.html"> |
| |
| Compatibility with Apache Hive |
| |
| </a> |
| </li> |
| |
| |
| |
| </ul> |
| |
| |
| |
| <li> |
| <a href="sql-reference.html"> |
| |
| Reference |
| |
| </a> |
| </li> |
| |
| |
| |
| </ul> |
| |
| </div> |
| </div> |
| |
| <input id="nav-trigger" class="nav-trigger" checked type="checkbox"> |
| <label for="nav-trigger"></label> |
| <div class="content-with-sidebar" id="content"> |
| |
| <h1 class="title">Spark SQL Upgrading Guide</h1> |
| |
| |
| <ul id="markdown-toc"> |
| <li><a href="#upgrading-from-spark-sql-24-to-245" id="markdown-toc-upgrading-from-spark-sql-24-to-245">Upgrading from Spark SQL 2.4 to 2.4.5</a></li> |
| <li><a href="#upgrading-from-spark-sql-243-to-244" id="markdown-toc-upgrading-from-spark-sql-243-to-244">Upgrading from Spark SQL 2.4.3 to 2.4.4</a></li> |
| <li><a href="#upgrading-from-spark-sql-24-to-241" id="markdown-toc-upgrading-from-spark-sql-24-to-241">Upgrading from Spark SQL 2.4 to 2.4.1</a></li> |
| <li><a href="#upgrading-from-spark-sql-23-to-24" id="markdown-toc-upgrading-from-spark-sql-23-to-24">Upgrading From Spark SQL 2.3 to 2.4</a></li> |
| <li><a href="#upgrading-from-spark-sql-230-to-231-and-above" id="markdown-toc-upgrading-from-spark-sql-230-to-231-and-above">Upgrading From Spark SQL 2.3.0 to 2.3.1 and above</a></li> |
| <li><a href="#upgrading-from-spark-sql-22-to-23" id="markdown-toc-upgrading-from-spark-sql-22-to-23">Upgrading From Spark SQL 2.2 to 2.3</a></li> |
| <li><a href="#upgrading-from-spark-sql-21-to-22" id="markdown-toc-upgrading-from-spark-sql-21-to-22">Upgrading From Spark SQL 2.1 to 2.2</a></li> |
| <li><a href="#upgrading-from-spark-sql-20-to-21" id="markdown-toc-upgrading-from-spark-sql-20-to-21">Upgrading From Spark SQL 2.0 to 2.1</a></li> |
| <li><a href="#upgrading-from-spark-sql-16-to-20" id="markdown-toc-upgrading-from-spark-sql-16-to-20">Upgrading From Spark SQL 1.6 to 2.0</a></li> |
| <li><a href="#upgrading-from-spark-sql-15-to-16" id="markdown-toc-upgrading-from-spark-sql-15-to-16">Upgrading From Spark SQL 1.5 to 1.6</a></li> |
| <li><a href="#upgrading-from-spark-sql-14-to-15" id="markdown-toc-upgrading-from-spark-sql-14-to-15">Upgrading From Spark SQL 1.4 to 1.5</a></li> |
| <li><a href="#upgrading-from-spark-sql-13-to-14" id="markdown-toc-upgrading-from-spark-sql-13-to-14">Upgrading from Spark SQL 1.3 to 1.4</a> <ul> |
| <li><a href="#dataframe-data-readerwriter-interface" id="markdown-toc-dataframe-data-readerwriter-interface">DataFrame data reader/writer interface</a></li> |
| <li><a href="#dataframegroupby-retains-grouping-columns" id="markdown-toc-dataframegroupby-retains-grouping-columns">DataFrame.groupBy retains grouping columns</a></li> |
| <li><a href="#behavior-change-on-dataframewithcolumn" id="markdown-toc-behavior-change-on-dataframewithcolumn">Behavior change on DataFrame.withColumn</a></li> |
| </ul> |
| </li> |
| <li><a href="#upgrading-from-spark-sql-10-12-to-13" id="markdown-toc-upgrading-from-spark-sql-10-12-to-13">Upgrading from Spark SQL 1.0-1.2 to 1.3</a> <ul> |
| <li><a href="#rename-of-schemardd-to-dataframe" id="markdown-toc-rename-of-schemardd-to-dataframe">Rename of SchemaRDD to DataFrame</a></li> |
| <li><a href="#unification-of-the-java-and-scala-apis" id="markdown-toc-unification-of-the-java-and-scala-apis">Unification of the Java and Scala APIs</a></li> |
| <li><a href="#isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only" id="markdown-toc-isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only">Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)</a></li> |
| <li><a href="#removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only" id="markdown-toc-removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only">Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)</a></li> |
| <li><a href="#udf-registration-moved-to-sqlcontextudf-java--scala" id="markdown-toc-udf-registration-moved-to-sqlcontextudf-java--scala">UDF Registration Moved to <code>sqlContext.udf</code> (Java & Scala)</a></li> |
| <li><a href="#python-datatypes-no-longer-singletons" id="markdown-toc-python-datatypes-no-longer-singletons">Python DataTypes No Longer Singletons</a></li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-24-to-245">Upgrading from Spark SQL 2.4 to 2.4.5</h2> |
| |
| <ul> |
| <li> |
| <p>Starting from 2.4.5, SQL configurations are effective also when a Dataset is converted to an RDD and its |
| plan is executed due to action on the derived RDD. The previous behavior can be restored setting |
| <code>spark.sql.legacy.rdd.applyConf</code> to <code>false</code>: in this case, SQL configurations are ignored for operations |
| performed on a RDD derived from a Dataset.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4.5, <code>TRUNCATE TABLE</code> command tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, set <code>spark.sql.truncateTable.ignorePermissionAcl.enabled</code> to <code>true</code>.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4.5, <code>spark.sql.legacy.mssqlserver.numericMapping.enabled</code> 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 <code>spark.sql.legacy.mssqlserver.numericMapping.enabled</code> to <code>true</code>.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-243-to-244">Upgrading from Spark SQL 2.4.3 to 2.4.4</h2> |
| |
| <pre><code>- 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. |
| </code></pre> |
| |
| <h2 id="upgrading-from-spark-sql-24-to-241">Upgrading from Spark SQL 2.4 to 2.4.1</h2> |
| |
| <ul> |
| <li>The value of <code>spark.executor.heartbeatInterval</code>, 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.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-23-to-24">Upgrading From Spark SQL 2.3 to 2.4</h2> |
| |
| <ul> |
| <li>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 <code>array_contains</code> 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.</li> |
| </ul> |
| <table class="table"> |
| <tr> |
| <th> |
| <b>Query</b> |
| </th> |
| <th> |
| <b>Result Spark 2.3 or Prior</b> |
| </th> |
| <th> |
| <b>Result Spark 2.4</b> |
| </th> |
| <th> |
| <b>Remarks</b> |
| </th> |
| </tr> |
| <tr> |
| <th> |
| <b>SELECT <br /> array_contains(array(1), 1.34D);</b> |
| </th> |
| <th> |
| <b>true</b> |
| </th> |
| <th> |
| <b>false</b> |
| </th> |
| <th> |
| <b>In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively.</b> |
| </th> |
| </tr> |
| <tr> |
| <th> |
| <b>SELECT <br /> array_contains(array(1), '1');</b> |
| </th> |
| <th> |
| <b>true</b> |
| </th> |
| <th> |
| <b>AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner.</b> |
| </th> |
| <th> |
| <b>Users can use explicit cast</b> |
| </th> |
| </tr> |
| <tr> |
| <th> |
| <b>SELECT <br /> array_contains(array(1), 'anystring');</b> |
| </th> |
| <th> |
| <b>null</b> |
| </th> |
| <th> |
| <b>AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner.</b> |
| </th> |
| <th> |
| <b>Users can use explicit cast</b> |
| </th> |
| </tr> |
| </table> |
| |
| <ul> |
| <li> |
| <p>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 <code>a</code> is a <code>struct(a string, b int)</code>, in Spark 2.4 <code>a in (select (1 as a, 'a' as b) from range(1))</code> is a valid query, while <code>a in (select 1, 'a' from range(1))</code> is not. In previous version it was the opposite.</p> |
| </li> |
| <li> |
| <p>In versions 2.2.1+ and 2.3, if <code>spark.sql.caseSensitive</code> is set to true, then the <code>CURRENT_DATE</code> and <code>CURRENT_TIMESTAMP</code> 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.</p> |
| </li> |
| <li> |
| <p>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 <code>spark.sql.legacy.setopsPrecedence.enabled</code> with a default value of <code>false</code>. When this property is set to <code>true</code>, 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.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, <code>spark.sql.orc.impl</code> and <code>spark.sql.orc.filterPushdown</code> change their default values to <code>native</code> and <code>true</code> respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use <code>spark.sql.orc.impl=hive</code> to create the files shared with Hive 2.1.1 and older.</p> |
| </li> |
| <li> |
| <p>In PySpark, when Arrow optimization is enabled, previously <code>toPandas</code> just failed when Arrow optimization is unable to be used whereas <code>createDataFrame</code> from Pandas DataFrame allowed the fallback to non-optimization. Now, both <code>toPandas</code> and <code>createDataFrame</code> from Pandas DataFrame allow the fallback by default, which can be switched off by <code>spark.sql.execution.arrow.fallback.enabled</code>.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>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 <code>UDF:f(col0 AS colA#28)</code> but <code>UDF:f(col0 AS `colA`)</code>.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set <code>false</code> to <code>spark.sql.legacy.compareDateTimestampInTimestamp</code> restores the previous behavior. This option will be removed in Spark 3.0.</p> |
| </li> |
| <li> |
| <p>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 <code>true</code> to <code>spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation</code> restores the previous behavior. This option will be removed in Spark 3.0.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like <code>TBLPROPERTIES (parquet.compression 'NONE')</code>. This happens for ORC Hive table properties like <code>TBLPROPERTIES (orc.compress 'NONE')</code> in case of <code>spark.sql.hive.convertMetastoreOrc=true</code>, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, <code>CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')</code> would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files.</p> |
| </li> |
| <li> |
| <p>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, <code>CREATE TABLE t(id int) STORED AS ORC</code> 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 <code>false</code> to <code>spark.sql.hive.convertMetastoreOrc</code> restores the previous behavior.</p> |
| </li> |
| <li> |
| <p>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 <code>spark.sql.csv.parser.columnPruning.enabled</code> to <code>false</code>.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting <code>spark.sql.statistics.parallelFileListingInStatsComputation.enabled</code> to <code>False</code>.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, empty strings are saved as quoted empty strings <code>""</code>. In version 2.3 and earlier, empty strings are equal to <code>null</code> values and do not reflect to any characters in saved CSV files. For example, the row of <code>"a", null, "", 1</code> was written as <code>a,,,1</code>. Since Spark 2.4, the same row is saved as <code>a,,"",1</code>. To restore the previous behavior, set the CSV option <code>emptyValue</code> to empty (not quoted) string.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.4, The LOAD DATA command supports wildcard <code>?</code> and <code>*</code>, which match any one character, and zero or more characters, respectively. Example: <code>LOAD DATA INPATH '/tmp/folder*/'</code> or <code>LOAD DATA INPATH '/tmp/part-?'</code>. Special Characters like <code>space</code> also now work in paths. Example: <code>LOAD DATA INPATH '/tmp/folder name/'</code>.</p> |
| </li> |
| <li> |
| <p>In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, <code>SELECT 1 FROM range(10) HAVING true</code> is executed as <code>SELECT 1 FROM range(10) WHERE true</code> 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 <code>SELECT 1 FROM range(10) HAVING true</code> will return only one row. To restore the previous behavior, set <code>spark.sql.legacy.parser.havingWithoutGroupByAsWhere</code> to <code>true</code>.</p> |
| </li> |
| <li> |
| <p>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 <code>spark.sql.caseSensitive</code> is set to <code>true</code> or <code>false</code>. Since 2.4, when <code>spark.sql.caseSensitive</code> is set to <code>false</code>, 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 <code>spark.sql.hive.convertMetastoreParquet</code> is set to <code>true</code>.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-230-to-231-and-above">Upgrading From Spark SQL 2.3.0 to 2.3.1 and above</h2> |
| |
| <ul> |
| <li>As of version 2.3.1 Arrow functionality, including <code>pandas_udf</code> and <code>toPandas()</code>/<code>createDataFrame()</code> with <code>spark.sql.execution.arrow.enabled</code> set to <code>True</code>, has been marked as experimental. These are still evolving and not currently recommended for use in production.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-22-to-23">Upgrading From Spark SQL 2.2 to 2.3</h2> |
| |
| <ul> |
| <li> |
| <p>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 <code>_corrupt_record</code> by default). For example, <code>spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()</code> and <code>spark.read.schema(schema).json(file).select("_corrupt_record").show()</code>. Instead, you can cache or save the parsed results and then send the same query. For example, <code>val df = spark.read.schema(schema).json(file).cache()</code> and then <code>df.filter($"_corrupt_record".isNotNull).count()</code>.</p> |
| </li> |
| <li> |
| <p>The <code>percentile_approx</code> 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.</p> |
| </li> |
| <li> |
| <p>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.</p> |
| </li> |
| <li>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: |
| <table class="table"> |
| <tr> |
| <th> |
| <b>InputA \ InputB</b> |
| </th> |
| <th> |
| <b>NullType</b> |
| </th> |
| <th> |
| <b>IntegerType</b> |
| </th> |
| <th> |
| <b>LongType</b> |
| </th> |
| <th> |
| <b>DecimalType(38,0)*</b> |
| </th> |
| <th> |
| <b>DoubleType</b> |
| </th> |
| <th> |
| <b>DateType</b> |
| </th> |
| <th> |
| <b>TimestampType</b> |
| </th> |
| <th> |
| <b>StringType</b> |
| </th> |
| </tr> |
| <tr> |
| <td> |
| <b>NullType</b> |
| </td> |
| <td>NullType</td> |
| <td>IntegerType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>DoubleType</td> |
| <td>DateType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>IntegerType</b> |
| </td> |
| <td>IntegerType</td> |
| <td>IntegerType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>LongType</b> |
| </td> |
| <td>LongType</td> |
| <td>LongType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DecimalType(38,0)*</b> |
| </td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DoubleType</b> |
| </td> |
| <td>DoubleType</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DateType</b> |
| </td> |
| <td>DateType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>DateType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>TimestampType</b> |
| </td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>TimestampType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>StringType</b> |
| </td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| </table> |
| |
| <p>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 <code>BigInteger</code>/<code>BigInt</code>. For example, 1.1 is inferred as double type.</p> |
| </li> |
| <li> |
| <p>In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as <code>toPandas</code>, <code>createDataFrame</code> from Pandas DataFrame, etc.</p> |
| </li> |
| <li> |
| <p>In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration <code>spark.sql.execution.pandas.respectSessionTimeZone</code> to <code>False</code>. See <a href="https://issues.apache.org/jira/browse/SPARK-22395">SPARK-22395</a> for details.</p> |
| </li> |
| <li> |
| <p>In PySpark, <code>na.fill()</code> or <code>fillna</code> also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame.</p> |
| </li> |
| <li> |
| <p>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 <a href="sql-performance-tuning.html#broadcast-hint-for-sql-queries">Broadcast Hint</a> and <a href="https://issues.apache.org/jira/browse/SPARK-22489">SPARK-22489</a>.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.3, when all inputs are binary, <code>functions.concat()</code> 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 <code>spark.sql.function.concatBinaryAsString</code> to <code>true</code>.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.3, when all inputs are binary, SQL <code>elt()</code> 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 <code>spark.sql.function.eltOutputAsString</code> to <code>true</code>.</p> |
| </li> |
| <li> |
| <p>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</p> |
| |
| <ul> |
| <li> |
| <p>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 (<code>+</code>), subtraction (<code>-</code>), multiplication (<code>*</code>), division (<code>/</code>), remainder (<code>%</code>) and positive module (<code>pmod</code>).</p> |
| </li> |
| <li> |
| <p>Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them.</p> |
| </li> |
| <li> |
| <p>The configuration <code>spark.sql.decimalOperations.allowPrecisionLoss</code> has been introduced. It defaults to <code>true</code>, which means the new behavior described here; if set to <code>false</code>, 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.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p>In PySpark, <code>df.replace</code> does not allow to omit <code>value</code> when <code>to_replace</code> is not a dictionary. Previously, <code>value</code> could be omitted in the other cases and had <code>None</code> by default, which is counterintuitive and error-prone.</p> |
| </li> |
| <li> |
| <p>Un-aliased subquery’s semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: <code>SELECT v.i from (SELECT i FROM v)</code>, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See <a href="https://issues.apache.org/jira/browse/SPARK-20690">SPARK-20690</a> and <a href="https://issues.apache.org/jira/browse/SPARK-21335">SPARK-21335</a> for more details.</p> |
| </li> |
| <li>When creating a <code>SparkSession</code> with <code>SparkSession.builder.getOrCreate()</code>, if there is an existing <code>SparkContext</code>, the builder was trying to update the <code>SparkConf</code> of the existing <code>SparkContext</code> with configurations specified to the builder, but the <code>SparkContext</code> is shared by all <code>SparkSession</code>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 <code>SparkSession</code>.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-21-to-22">Upgrading From Spark SQL 2.1 to 2.2</h2> |
| |
| <ul> |
| <li> |
| <p>Spark 2.1.1 introduced a new configuration key: <code>spark.sql.hive.caseSensitiveInferenceMode</code>. It had a default setting of <code>NEVER_INFER</code>, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting’s default value to <code>INFER_AND_SAVE</code> to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the <code>INFER_AND_SAVE</code> 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 <code>spark.sql.hive.caseSensitiveInferenceMode</code> to <code>NEVER_INFER</code> to avoid the initial overhead of schema inference. Note that with the new default <code>INFER_AND_SAVE</code> 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.</p> |
| </li> |
| <li> |
| <p>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).</p> |
| </li> |
| <li> |
| <p>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 <code>ALTER VIEW AS</code> or <code>CREATE OR REPLACE VIEW AS</code> with newer Spark versions.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-20-to-21">Upgrading From Spark SQL 2.0 to 2.1</h2> |
| |
| <ul> |
| <li> |
| <p>Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as <code>ALTER TABLE PARTITION ... SET LOCATION</code> are now available for tables created with the Datasource API.</p> |
| |
| <ul> |
| <li> |
| <p>Legacy datasource tables can be migrated to this format via the <code>MSCK REPAIR TABLE</code> command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance.</p> |
| </li> |
| <li> |
| <p>To determine if a table has been migrated, look for the <code>PartitionProvider: Catalog</code> attribute when issuing <code>DESCRIBE FORMATTED</code> on the table.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p>Changes to <code>INSERT OVERWRITE TABLE ... PARTITION ...</code> behavior for Datasource tables.</p> |
| |
| <ul> |
| <li> |
| <p>In prior Spark versions <code>INSERT OVERWRITE</code> overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten.</p> |
| </li> |
| <li> |
| <p>Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data.</p> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-16-to-20">Upgrading From Spark SQL 1.6 to 2.0</h2> |
| |
| <ul> |
| <li> |
| <p><code>SparkSession</code> is now the new entry point of Spark that replaces the old <code>SQLContext</code> and</p> |
| |
| <p><code>HiveContext</code>. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new <code>catalog</code> interface is accessible from <code>SparkSession</code> - existing API on databases and tables access such as <code>listTables</code>, <code>createExternalTable</code>, <code>dropTempView</code>, <code>cacheTable</code> are moved here.</p> |
| </li> |
| <li> |
| <p>Dataset API and DataFrame API are unified. In Scala, <code>DataFrame</code> becomes a type alias for |
| <code>Dataset[Row]</code>, while Java API users must replace <code>DataFrame</code> with <code>Dataset<Row></code>. Both the typed |
| transformations (e.g., <code>map</code>, <code>filter</code>, and <code>groupByKey</code>) and untyped transformations (e.g., |
| <code>select</code> and <code>groupBy</code>) 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, <code>DataFrame</code> remains the primary programming abstraction, which is analogous to the |
| single-node data frame notion in these languages.</p> |
| </li> |
| <li> |
| <p>Dataset and DataFrame API <code>unionAll</code> has been deprecated and replaced by <code>union</code></p> |
| </li> |
| <li> |
| <p>Dataset and DataFrame API <code>explode</code> has been deprecated, alternatively, use <code>functions.explode()</code> with <code>select</code> or <code>flatMap</code></p> |
| </li> |
| <li> |
| <p>Dataset and DataFrame API <code>registerTempTable</code> has been deprecated and replaced by <code>createOrReplaceTempView</code></p> |
| </li> |
| <li> |
| <p>Changes to <code>CREATE TABLE ... LOCATION</code> behavior for Hive tables.</p> |
| |
| <ul> |
| <li> |
| <p>From Spark 2.0, <code>CREATE TABLE ... LOCATION</code> is equivalent to <code>CREATE EXTERNAL TABLE ... LOCATION</code> |
| in 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.</p> |
| </li> |
| <li> |
| <p>As a result, <code>DROP TABLE</code> statements on those tables will not remove the data.</p> |
| </li> |
| </ul> |
| </li> |
| <li> |
| <p><code>spark.sql.parquet.cacheMetadata</code> is no longer used. |
| See <a href="https://issues.apache.org/jira/browse/SPARK-13664">SPARK-13664</a> for details.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-15-to-16">Upgrading From Spark SQL 1.5 to 1.6</h2> |
| |
| <ul> |
| <li>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 <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add |
| this option to <code>spark-defaults.conf</code>, or pass it to <code>start-thriftserver.sh</code> via <code>--conf</code>:</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span></span> ./sbin/start-thriftserver.sh <span class="se">\</span> |
| --conf spark.sql.hive.thriftServer.singleSession<span class="o">=</span><span class="nb">true</span> <span class="se">\</span> |
| ... |
| </code></pre></figure> |
| |
| <ul> |
| <li> |
| <p>Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns |
| of the same name of a DataFrame.</p> |
| </li> |
| <li> |
| <p>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 <a href="https://issues.apache.org/jira/browse/SPARK-11724">SPARK-11724</a> for |
| details.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-14-to-15">Upgrading From Spark SQL 1.4 to 1.5</h2> |
| |
| <ul> |
| <li> |
| <p>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 |
| <code>spark.sql.tungsten.enabled</code> to <code>false</code>.</p> |
| </li> |
| <li> |
| <p>Parquet schema merging is no longer enabled by default. It can be re-enabled by setting |
| <code>spark.sql.parquet.mergeSchema</code> to <code>true</code>.</p> |
| </li> |
| <li> |
| <p>Resolution of strings to columns in python now supports using dots (<code>.</code>) to qualify the column or |
| access nested values. For example <code>df['table.column.nestedField']</code>. However, this means that if |
| your column name contains any dots you must now escape them using backticks (e.g., <code>table.`column.with.dots`.nested</code>).</p> |
| </li> |
| <li> |
| <p>In-memory columnar storage partition pruning is on by default. It can be disabled by setting |
| <code>spark.sql.inMemoryColumnarStorage.partitionPruning</code> to <code>false</code>.</p> |
| </li> |
| <li> |
| <p>Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum |
| precision of 38. When inferring schema from <code>BigDecimal</code> objects, a precision of (38, 18) is now |
| used. When no precision is specified in DDL then the default remains <code>Decimal(10, 0)</code>.</p> |
| </li> |
| <li> |
| <p>Timestamps are now stored at a precision of 1us, rather than 1ns</p> |
| </li> |
| <li> |
| <p>In the <code>sql</code> dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains |
| unchanged.</p> |
| </li> |
| <li> |
| <p>The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM).</p> |
| </li> |
| <li> |
| <p>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 <code>REFRESH TABLE</code> SQL command or <code>HiveContext</code>’s <code>refreshTable</code> method |
| 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.</p> |
| </li> |
| <li> |
| <p>DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-13-to-14">Upgrading from Spark SQL 1.3 to 1.4</h2> |
| |
| <h4 id="dataframe-data-readerwriter-interface">DataFrame data reader/writer interface</h4> |
| |
| <p>Based on user feedback, we created a new, more fluid API for reading data in (<code>SQLContext.read</code>) |
| and writing data out (<code>DataFrame.write</code>), |
| and deprecated the old APIs (e.g., <code>SQLContext.parquetFile</code>, <code>SQLContext.jsonFile</code>).</p> |
| |
| <p>See the API docs for <code>SQLContext.read</code> ( |
| <a href="api/scala/index.html#org.apache.spark.sql.SQLContext@read:DataFrameReader">Scala</a>, |
| <a href="api/java/org/apache/spark/sql/SQLContext.html#read()">Java</a>, |
| <a href="api/python/pyspark.sql.html#pyspark.sql.SQLContext.read">Python</a> |
| ) and <code>DataFrame.write</code> ( |
| <a href="api/scala/index.html#org.apache.spark.sql.DataFrame@write:DataFrameWriter">Scala</a>, |
| <a href="api/java/org/apache/spark/sql/Dataset.html#write()">Java</a>, |
| <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame.write">Python</a> |
| ) more information.</p> |
| |
| <h4 id="dataframegroupby-retains-grouping-columns">DataFrame.groupBy retains grouping columns</h4> |
| |
| <p>Based on user feedback, we changed the default behavior of <code>DataFrame.groupBy().agg()</code> to retain the |
| grouping columns in the resulting <code>DataFrame</code>. To keep the behavior in 1.3, set <code>spark.sql.retainGroupColumns</code> to <code>false</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="c1">// In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1">// it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="n">agg</span><span class="o">(</span><span class="n">$</span><span class="s">"department"</span><span class="o">,</span> <span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">))</span> |
| |
| <span class="c1">// In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="n">agg</span><span class="o">(</span><span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">))</span> |
| |
| <span class="c1">// Revert to 1.3 behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="o">(</span><span class="s">"spark.sql.retainGroupColumns"</span><span class="o">,</span> <span class="s">"false"</span><span class="o">)</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="c1">// In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1">// it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="na">agg</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"department"</span><span class="o">),</span> <span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">));</span> |
| |
| <span class="c1">// In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="na">agg</span><span class="o">(</span><span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">));</span> |
| |
| <span class="c1">// Revert to 1.3 behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="na">setConf</span><span class="o">(</span><span class="s">"spark.sql.retainGroupColumns"</span><span class="o">,</span> <span class="s">"false"</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span></span><span class="kn">import</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">as</span> <span class="nn">func</span> |
| |
| <span class="c1"># In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1"># it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s2">"department"</span><span class="p">)</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">df</span><span class="p">[</span><span class="s2">"department"</span><span class="p">],</span> <span class="n">func</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="s2">"age"</span><span class="p">),</span> <span class="n">func</span><span class="o">.</span><span class="n">sum</span><span class="p">(</span><span class="s2">"expense"</span><span class="p">))</span> |
| |
| <span class="c1"># In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s2">"department"</span><span class="p">)</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="s2">"age"</span><span class="p">),</span> <span class="n">func</span><span class="o">.</span><span class="n">sum</span><span class="p">(</span><span class="s2">"expense"</span><span class="p">))</span> |
| |
| <span class="c1"># Revert to 1.3.x behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="p">(</span><span class="s2">"spark.sql.retainGroupColumns"</span><span class="p">,</span> <span class="s2">"false"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h4 id="behavior-change-on-dataframewithcolumn">Behavior change on DataFrame.withColumn</h4> |
| |
| <p>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.</p> |
| |
| <p>Note that this change is only for Scala API, not for PySpark and SparkR.</p> |
| |
| <h2 id="upgrading-from-spark-sql-10-12-to-13">Upgrading from Spark SQL 1.0-1.2 to 1.3</h2> |
| |
| <p>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).</p> |
| |
| <h4 id="rename-of-schemardd-to-dataframe">Rename of SchemaRDD to DataFrame</h4> |
| |
| <p>The largest change that users will notice when upgrading to Spark SQL 1.3 is that <code>SchemaRDD</code> has |
| been renamed to <code>DataFrame</code>. 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 <code>.rdd</code> method.</p> |
| |
| <p>In Scala, there is a type alias from <code>SchemaRDD</code> to <code>DataFrame</code> to provide source compatibility for |
| some use cases. It is still recommended that users update their code to use <code>DataFrame</code> instead. |
| Java and Python users will need to update their code.</p> |
| |
| <h4 id="unification-of-the-java-and-scala-apis">Unification of the Java and Scala APIs</h4> |
| |
| <p>Prior to Spark 1.3 there were separate Java compatible classes (<code>JavaSQLContext</code> and <code>JavaSchemaRDD</code>) |
| that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users |
| of either language should use <code>SQLContext</code> and <code>DataFrame</code>. In general these classes try to |
| use types that are usable from both languages (i.e. <code>Array</code> 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.</p> |
| |
| <p>Additionally, the Java specific types API has been removed. Users of both Scala and Java should |
| use the classes present in <code>org.apache.spark.sql.types</code> to describe schema programmatically.</p> |
| |
| <h4 id="isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only">Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)</h4> |
| |
| <p>Many of the code examples prior to Spark 1.3 started with <code>import sqlContext._</code>, which brought |
| all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit |
| conversions for converting <code>RDD</code>s into <code>DataFrame</code>s into an object inside of the <code>SQLContext</code>. |
| Users should now write <code>import sqlContext.implicits._</code>.</p> |
| |
| <p>Additionally, the implicit conversions now only augment RDDs that are composed of <code>Product</code>s (i.e., |
| case classes or tuples) with a method <code>toDF</code>, instead of applying automatically.</p> |
| |
| <p>When using function inside of the DSL (now replaced with the <code>DataFrame</code> API) users used to import |
| <code>org.apache.spark.sql.catalyst.dsl</code>. Instead the public dataframe functions API should be used: |
| <code>import org.apache.spark.sql.functions._</code>.</p> |
| |
| <h4 id="removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only">Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)</h4> |
| |
| <p>Spark 1.3 removes the type aliases that were present in the base sql package for <code>DataType</code>. Users |
| should instead import the classes in <code>org.apache.spark.sql.types</code></p> |
| |
| <h4 id="udf-registration-moved-to-sqlcontextudf-java--scala">UDF Registration Moved to <code>sqlContext.udf</code> (Java & Scala)</h4> |
| |
| <p>Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been |
| moved into the udf object in <code>SQLContext</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="n">sqlContext</span><span class="o">.</span><span class="n">udf</span><span class="o">.</span><span class="n">register</span><span class="o">(</span><span class="s">"strLen"</span><span class="o">,</span> <span class="o">(</span><span class="n">s</span><span class="k">:</span> <span class="kt">String</span><span class="o">)</span> <span class="k">=></span> <span class="n">s</span><span class="o">.</span><span class="n">length</span><span class="o">())</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="n">sqlContext</span><span class="o">.</span><span class="na">udf</span><span class="o">().</span><span class="na">register</span><span class="o">(</span><span class="s">"strLen"</span><span class="o">,</span> <span class="o">(</span><span class="n">String</span> <span class="n">s</span><span class="o">)</span> <span class="o">-></span> <span class="n">s</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">IntegerType</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>Python UDF registration is unchanged.</p> |
| |
| <h4 id="python-datatypes-no-longer-singletons">Python DataTypes No Longer Singletons</h4> |
| |
| <p>When using DataTypes in Python you will need to construct them (i.e. <code>StringType()</code>) instead of |
| referencing a singleton.</p> |
| |
| |
| </div> |
| |
| <!-- /container --> |
| </div> |
| |
| <script src="js/vendor/jquery-1.12.4.min.js"></script> |
| <script src="js/vendor/bootstrap.min.js"></script> |
| <script src="js/vendor/anchor.min.js"></script> |
| <script src="js/main.js"></script> |
| |
| <!-- MathJax Section --> |
| <script type="text/x-mathjax-config"> |
| MathJax.Hub.Config({ |
| TeX: { equationNumbers: { autoNumber: "AMS" } } |
| }); |
| </script> |
| <script> |
| // Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS. |
| // We could use "//cdn.mathjax...", but that won't support "file://". |
| (function(d, script) { |
| script = d.createElement('script'); |
| script.type = 'text/javascript'; |
| script.async = true; |
| script.onload = function(){ |
| MathJax.Hub.Config({ |
| tex2jax: { |
| inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ], |
| displayMath: [ ["$$","$$"], ["\\[", "\\]"] ], |
| processEscapes: true, |
| skipTags: ['script', 'noscript', 'style', 'textarea', 'pre'] |
| } |
| }); |
| }; |
| script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + |
| 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + |
| '?config=TeX-AMS-MML_HTMLorMML'; |
| d.getElementsByTagName('head')[0].appendChild(script); |
| }(document)); |
| </script> |
| </body> |
| </html> |