e0b20f9f24
## What changes were proposed in this pull request? Currently `MapObjects` does not make copies of unsafe-backed data, leading to problems like [SPARK-17061](https://issues.apache.org/jira/browse/SPARK-17061) [SPARK-17093](https://issues.apache.org/jira/browse/SPARK-17093). This patch makes `MapObjects` make copies of unsafe-backed data. Generated code - prior to this patch: ```java ... /* 295 */ if (isNull12) { /* 296 */ convertedArray1[loopIndex1] = null; /* 297 */ } else { /* 298 */ convertedArray1[loopIndex1] = value12; /* 299 */ } ... ``` Generated code - after this patch: ```java ... /* 295 */ if (isNull12) { /* 296 */ convertedArray1[loopIndex1] = null; /* 297 */ } else { /* 298 */ convertedArray1[loopIndex1] = value12 instanceof UnsafeRow? value12.copy() : value12; /* 299 */ } ... ``` ## How was this patch tested? Add a new test case which would fail without this patch. Author: Liwei Lin <lwlin7@gmail.com> Closes #14698 from lw-lin/mapobjects-copy. |
||
---|---|---|
.. | ||
catalyst | ||
core | ||
hive | ||
hive-thriftserver | ||
README.md |
Spark SQL
This module provides support for executing relational queries expressed in either SQL or the DataFrame/Dataset API.
Spark SQL is broken up into four subprojects:
- Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions.
- Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files.
- Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs.
- HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server.