Commit graph

2196 commits

Author SHA1 Message Date
Wenchen Fan 42d225f449 [SPARK-11216][SQL][FOLLOW-UP] add encoder/decoder for external row
address comments in https://github.com/apache/spark/pull/9184

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9212 from cloud-fan/encoder.
2015-10-22 10:53:59 -07:00
Josh Rosen f6d06adf05 [SPARK-10708] Consolidate sort shuffle implementations
There's a lot of duplication between SortShuffleManager and UnsafeShuffleManager. Given that these now provide the same set of functionality, now that UnsafeShuffleManager supports large records, I think that we should replace SortShuffleManager's serialized shuffle implementation with UnsafeShuffleManager's and should merge the two managers together.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8829 from JoshRosen/consolidate-sort-shuffle-implementations.
2015-10-22 09:46:30 -07:00
Davies Liu 1d97332715 [SPARK-11243][SQL] output UnsafeRow from columnar cache
This PR change InMemoryTableScan to output UnsafeRow, and optimize the unrolling and scanning by coping the bytes for var-length types between UnsafeRow and ByteBuffer directly without creating the wrapper objects. When scanning the decimals in TPC-DS store_sales table, it's 80% faster (copy it as long without create Decimal objects).

Author: Davies Liu <davies@databricks.com>

Closes #9203 from davies/unsafe_cache.
2015-10-21 19:20:31 -07:00
Yanbo Liang 40a10d7675 [SPARK-9392][SQL] Dataframe drop should work on unresolved columns
Dataframe drop should work on unresolved columns

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8821 from yanboliang/spark-9392.
2015-10-21 17:50:33 -07:00
navis.ryu f481090a71 [SPARK-10151][SQL] Support invocation of hive macro
Macro in hive (which is GenericUDFMacro) contains real function inside of it but it's not conveyed to tasks, resulting null-pointer exception.

Author: navis.ryu <navis@apache.org>

Closes #8354 from navis/SPARK-10151.
2015-10-21 15:07:08 -07:00
Dilip Biswal dce2f8c9d7 [SPARK-8654][SQL] Analysis exception when using NULL IN (...) : invalid cast
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.

The fix is to find a common type between LHS and RHS expressions and cast
all the expression to the common type.

Author: Dilip Biswal <dbiswal@us.ibm.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #9036 from dilipbiswal/spark_8654_new.
2015-10-21 14:29:03 -07:00
Shagun Sodhani 19ad18638e [SPARK-11233][SQL] register cosh in function registry
Author: Shagun Sodhani <sshagunsodhani@gmail.com>

Closes #9199 from shagunsodhani/proposed-fix-#11233.
2015-10-21 14:18:06 -07:00
Artem Aliev a37cd87048 [SPARK-11208][SQL] Filter out 'hive.metastore.rawstore.impl' from executionHive temporary config
The executionHive assumed to be a standard meta store located in temporary directory as a derby db. But hive.metastore.rawstore.impl was not filtered out so any custom implementation of the metastore with other storage properties (not JDO) will persist that temporary functions. CassandraHiveMetaStore from DataStax Enterprise is one of examples.

Author: Artem Aliev <artem.aliev@datastax.com>

Closes #9178 from artem-aliev/SPARK-11208.
2015-10-21 14:12:07 -07:00
Yin Huai 3afe448d39 [SPARK-9740][SPARK-9592][SPARK-9210][SQL] Change the default behavior of First/Last to RESPECT NULLS.
I am changing the default behavior of `First`/`Last` to respect null values (the SQL standard default behavior).

https://issues.apache.org/jira/browse/SPARK-9740

Author: Yin Huai <yhuai@databricks.com>

Closes #8113 from yhuai/firstLast.
2015-10-21 13:43:17 -07:00
Davies Liu f8c6bec657 [SPARK-11197][SQL] run SQL on files directly
This PR introduce a new feature to run SQL directly on files without create a table, for example:

```
select id from json.`path/to/json/files` as j
```

Author: Davies Liu <davies@databricks.com>

Closes #9173 from davies/source.
2015-10-21 13:38:30 -07:00
Wenchen Fan 7c74ebca05 [SPARK-10743][SQL] keep the name of expression if possible when do cast
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8859 from cloud-fan/cast.
2015-10-21 13:22:35 -07:00
Dilip Biswal 49ea0e9d7c [SPARK-10534] [SQL] ORDER BY clause allows only columns that are present in the select projection list
Find out the missing attributes by recursively looking
at the sort order expression and rest of the code
takes care of projecting them out.

Added description from cloud-fan

I wanna explain a bit more about this bug.

When we resolve sort ordering, we will use a special method, which only resolves UnresolvedAttributes and UnresolvedExtractValue. However, for something like Floor('a), even the 'a is resolved, the floor expression may still being unresolved as data type mismatch(for example, 'a is string type and Floor need double type), thus can't pass this filter, and we can't push down this missing attribute 'a

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #9123 from dilipbiswal/SPARK-10534.
2015-10-21 11:10:32 -07:00
Wenchen Fan ccf536f903 [SPARK-11216] [SQL] add encoder/decoder for external row
Implement encode/decode for external row based on `ClassEncoder`.

TODO:
* code cleanup
* ~~fix corner cases~~
* refactor the encoder interface
* improve test for product codegen, to cover more corner cases.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9184 from cloud-fan/encoder.
2015-10-21 11:06:34 -07:00
nitin goyal f62e326088 [SPARK-11179] [SQL] Push filters through aggregate
Push conjunctive predicates though Aggregate operators when their references are a subset of the groupingExpressions.

Query plan before optimisation :-
Filter ((c#138L = 2) && (a#0 = 3))
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Project [a#0,b#1]
   LocalRelation [a#0,b#1,c#2]

Query plan after optimisation :-
Filter (c#138L = 2)
 Aggregate [a#0], [a#0,count(b#1) AS c#138L]
  Filter (a#0 = 3)
   Project [a#0,b#1]
    LocalRelation [a#0,b#1,c#2]

Author: nitin goyal <nitin.goyal@guavus.com>
Author: nitin.goyal <nitin.goyal@guavus.com>

Closes #9167 from nitin2goyal/master.
2015-10-21 10:45:21 -07:00
Pravin Gadakh 8e82e59834 [SPARK-11037][SQL] using Option instead of Some in JdbcDialects
Using Option instead of Some in getCatalystType method.

Author: Pravin Gadakh <prgadakh@in.ibm.com>

Closes #9195 from pravingadakh/master.
2015-10-21 10:35:09 -07:00
Cheng Lian 89e6db6150 [SPARK-11153][SQL] Disables Parquet filter push-down for string and binary columns
Due to PARQUET-251, `BINARY` columns in existing Parquet files may be written with corrupted statistics information. This information is used by filter push-down optimization. Since Spark 1.5 turns on Parquet filter push-down by default, we may end up with wrong query results. PARQUET-251 has been fixed in parquet-mr 1.8.1, but Spark 1.5 is still using 1.7.0.

This affects all Spark SQL data types that can be mapped to Parquet {{BINARY}}, namely:

- `StringType`

- `BinaryType`

- `DecimalType`

  (But Spark SQL doesn't support pushing down filters involving `DecimalType` columns for now.)

To avoid wrong query results, we should disable filter push-down for columns of `StringType` and `BinaryType` until we upgrade to parquet-mr 1.8.

Author: Cheng Lian <lian@databricks.com>

Closes #9152 from liancheng/spark-11153.workaround-parquet-251.

(cherry picked from commit 0887e5e878)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-10-21 09:02:59 +08:00
Davies Liu 06e6b765d0 [SPARK-11149] [SQL] Improve cache performance for primitive types
This PR improve the performance by:

1) Generate an Iterator that take Iterator[CachedBatch] as input, and call accessors (unroll the loop for columns), avoid the expensive Iterator.flatMap.

2) Use Unsafe.getInt/getLong/getFloat/getDouble instead of ByteBuffer.getInt/getLong/getFloat/getDouble, the later one actually read byte by byte.

3) Remove the unnecessary copy() in Coalesce(), which is not related to memory cache, found during benchmark.

The following benchmark showed that we can speedup the columnar cache of int by 2x.

```
path = '/opt/tpcds/store_sales/'
int_cols = ['ss_sold_date_sk', 'ss_sold_time_sk', 'ss_item_sk','ss_customer_sk']
df = sqlContext.read.parquet(path).select(int_cols).cache()
df.count()

t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```

Author: Davies Liu <davies@databricks.com>

Closes #9145 from davies/byte_buffer.
2015-10-20 14:01:53 -07:00
Davies Liu 67d468f8d9 [SPARK-11111] [SQL] fast null-safe join
Currently, we use CartesianProduct for join with null-safe-equal condition.
```
scala> sqlContext.sql("select * from t a join t b on (a.i <=> b.i)").explain
== Physical Plan ==
TungstenProject [i#2,j#3,i#7,j#8]
 Filter (i#2 <=> i#7)
  CartesianProduct
   LocalTableScan [i#2,j#3], [[1,1]]
   LocalTableScan [i#7,j#8], [[1,1]]
```
Actually, we can have an equal-join condition as  `coalesce(i, default) = coalesce(b.i, default)`, then an partitioned join algorithm could be used.

After this PR, the plan will become:
```
>>> sqlContext.sql("select * from a join b ON a.id <=> b.id").explain()
TungstenProject [id#0L,id#1L]
 Filter (id#0L <=> id#1L)
  SortMergeJoin [coalesce(id#0L,0)], [coalesce(id#1L,0)]
   TungstenSort [coalesce(id#0L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#0L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#0L]
   TungstenSort [coalesce(id#1L,0) ASC], false, 0
    TungstenExchange hashpartitioning(coalesce(id#1L,0),200)
     ConvertToUnsafe
      Scan PhysicalRDD[id#1L]
```

Author: Davies Liu <davies@databricks.com>

Closes #9120 from davies/null_safe.
2015-10-20 13:40:24 -07:00
Wenchen Fan 478c7ce862 [SPARK-6740] [SQL] correctly parse NOT operator with comparison operations
We can't parse `NOT` operator with comparison operations like `SELECT NOT TRUE > TRUE`, this PR fixed it.

Takes over https://github.com/apache/spark/pull/6326.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #8617 from cloud-fan/not.
2015-10-20 13:38:25 -07:00
Daoyuan Wang 94139557c5 [SPARK-10463] [SQL] remove PromotePrecision during optimization
PromotePrecision is not necessary after HiveTypeCoercion done.

Jira:
https://issues.apache.org/jira/browse/SPARK-10463

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #8621 from adrian-wang/promoterm.
2015-10-20 09:20:35 -07:00
Jakob Odersky 60851bc7bf [SPARK-11110][BUILD] Remove transient annotation for parameters.
`transient` annotations on class parameters (not case class parameters or vals) causes compilation errors during compilation with Scala 2.11.
I understand that transient *parameters* make no sense, however I don't quite understand why the 2.10 compiler accepted them.

Note: in case it is preferred to keep the annotations in case someone would in the future want to redefine them as vals, it would also be possible to just add `val` after the annotation, e.g. `class Foo(transient x: Int)` becomes `class Foo(transient private val x: Int)`.

I chose to remove the annotation as it also reduces needles clutter, however please feel free to tell me if you prefer the second option and I'll update the PR

Author: Jakob Odersky <jodersky@gmail.com>

Closes #9126 from jodersky/sbt-scala-2.11.
2015-10-20 08:54:34 +01:00
Cheng Lian 8b877cc4ee [SPARK-11088][SQL] Merges partition values using UnsafeProjection
`DataSourceStrategy.mergeWithPartitionValues` is essentially a projection implemented in a quite inefficient way. This PR optimizes this method with `UnsafeProjection` to avoid unnecessary boxing costs.

Author: Cheng Lian <lian@databricks.com>

Closes #9104 from liancheng/spark-11088.faster-partition-values-merging.
2015-10-19 16:57:20 -07:00
Rishabh Bhardwaj 5966817941 [SPARK-11180][SQL] Support BooleanType in DataFrame.na.fill
Added support for boolean types in fill and replace methods

Author: Rishabh Bhardwaj <rbnext29@gmail.com>

Closes #9166 from rishabhbhardwaj/master.
2015-10-19 14:38:58 -07:00
Wenchen Fan 7893cd95db [SPARK-11119] [SQL] cleanup for unsafe array and map
The purpose of this PR is to keep the unsafe format detail only inside the unsafe class itself, so when we use them(like use unsafe array in unsafe map, use unsafe array and map in columnar cache), we don't need to understand the format before use them.

change list:
* unsafe array's 4-bytes numElements header is now required(was optional), and become a part of unsafe array format.
* w.r.t the previous changing, the `sizeInBytes` of unsafe array now counts the 4-bytes header.
* unsafe map's format was `[numElements] [key array numBytes] [key array content(without numElements header)] [value array content(without numElements header)]` before, which is a little hacky as it makes unsafe array's header optional. I think saving 4 bytes is not a big deal, so the format is now: `[key array numBytes] [unsafe key array] [unsafe value array]`.
* w.r.t the previous changing, the `sizeInBytes` of unsafe map now counts both map's header and array's header.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9131 from cloud-fan/unsafe.
2015-10-19 11:02:26 -07:00
zsxwing beb8bc1ea5 [SPARK-11126][SQL] Fix the potential flaky test
The unit test added in #9132 is flaky. This is a follow up PR to add `listenerBus.waitUntilEmpty` to fix it.

Author: zsxwing <zsxwing@gmail.com>

Closes #9163 from zsxwing/SPARK-11126-follow-up.
2015-10-19 00:06:51 -07:00
zsxwing 94c8fef296 [SPARK-11126][SQL] Fix a memory leak in SQLListener._stageIdToStageMetrics
SQLListener adds all stage infos to `_stageIdToStageMetrics`, but only removes stage infos belonging to SQL executions. This PR fixed it by ignoring stages that don't belong to SQL executions.

Reported by Terry Hoo in https://www.mail-archive.com/userspark.apache.org/msg38810.html

Author: zsxwing <zsxwing@gmail.com>

Closes #9132 from zsxwing/SPARK-11126.
2015-10-18 13:51:45 -07:00
tedyu 3895b2113a [SPARK-11172] Close JsonParser/Generator in test
Author: tedyu <yuzhihong@gmail.com>

Closes #9157 from tedyu/master.
2015-10-18 02:12:56 -07:00
Koert Kuipers 57f83e36d6 [SPARK-10185] [SQL] Feat sql comma separated paths
Make sure comma-separated paths get processed correcly in ResolvedDataSource for a HadoopFsRelationProvider

Author: Koert Kuipers <koert@tresata.com>

Closes #8416 from koertkuipers/feat-sql-comma-separated-paths.
2015-10-17 14:56:24 -07:00
Pravin Gadakh 3d683a139b [SPARK-10581] [DOCS] Groups are not resolved in scaladoc in sql classes
Groups are not resolved properly in scaladoc in following classes:

sql/core/src/main/scala/org/apache/spark/sql/Column.scala
sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Author: Pravin Gadakh <pravingadakh177@gmail.com>

Closes #9148 from pravingadakh/master.
2015-10-16 13:38:50 -07:00
navis.ryu b9c5e5d4ac [SPARK-11124] JsonParser/Generator should be closed for resource recycle
Some json parsers are not closed. parser in JacksonParser#parseJson, for example.

Author: navis.ryu <navis@apache.org>

Closes #9130 from navis/SPARK-11124.
2015-10-16 11:19:37 -07:00
Josh Rosen eb0b4d6e2d [SPARK-11135] [SQL] Exchange incorrectly skips sorts when existing ordering is non-empty subset of required ordering
In Spark SQL, the Exchange planner tries to avoid unnecessary sorts in cases where the data has already been sorted by a superset of the requested sorting columns. For instance, let's say that a query calls for an operator's input to be sorted by `a.asc` and the input happens to already be sorted by `[a.asc, b.asc]`. In this case, we do not need to re-sort the input. The converse, however, is not true: if the query calls for `[a.asc, b.asc]`, then `a.asc` alone will not satisfy the ordering requirements, requiring an additional sort to be planned by Exchange.

However, the current Exchange code gets this wrong and incorrectly skips sorting when the existing output ordering is a subset of the required ordering. This is simple to fix, however.

This bug was introduced in https://github.com/apache/spark/pull/7458, so it affects 1.5.0+.

This patch fixes the bug and significantly improves the unit test coverage of Exchange's sort-planning logic.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9140 from JoshRosen/SPARK-11135.
2015-10-15 17:36:55 -07:00
Wenchen Fan 6a2359ff1f [SPARK-10412] [SQL] report memory usage for tungsten sql physical operator
https://issues.apache.org/jira/browse/SPARK-10412

some screenshots:
### aggregate:
![screen shot 2015-10-12 at 2 23 11 pm](https://cloud.githubusercontent.com/assets/3182036/10439534/618320a4-70ef-11e5-94d8-62ea7f2d1531.png)

### join
![screen shot 2015-10-12 at 2 23 29 pm](https://cloud.githubusercontent.com/assets/3182036/10439537/6724797c-70ef-11e5-8f75-0cf5cbd42048.png)

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8931 from cloud-fan/viz.
2015-10-15 14:50:58 -07:00
Andrew Or 3b364ff0a4 [SPARK-11078] Ensure spilling tests actually spill
#9084 uncovered that many tests that test spilling don't actually spill. This is a follow-up patch to fix that to ensure our unit tests actually catch potential bugs in spilling. The size of this patch is inflated by the refactoring of `ExternalSorterSuite`, which had a lot of duplicate code and logic.

Author: Andrew Or <andrew@databricks.com>

Closes #9124 from andrewor14/spilling-tests.
2015-10-15 14:50:01 -07:00
Cheng Hao 9808052b5a [SPARK-11076] [SQL] Add decimal support for floor and ceil
Actually all of the `UnaryMathExpression` doens't support the Decimal, will create follow ups for supporing it. This is the first PR which will be good to review the approach I am taking.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #9086 from chenghao-intel/ceiling.
2015-10-14 20:56:08 -07:00
Josh Rosen 4ace4f8a9c [SPARK-11017] [SQL] Support ImperativeAggregates in TungstenAggregate
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.

The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
2015-10-14 17:27:50 -07:00
Cheng Hao 1baaf2b9bd [SPARK-10829] [SQL] Filter combine partition key and attribute doesn't work in DataSource scan
```scala
withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") {
      withTempPath { dir =>
        val path = s"${dir.getCanonicalPath}/part=1"
        (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path)

        // If the "part = 1" filter gets pushed down, this query will throw an exception since
        // "part" is not a valid column in the actual Parquet file
        checkAnswer(
          sqlContext.read.parquet(path).filter("a > 0 and (part = 0 or a > 1)"),
          (2 to 3).map(i => Row(i, i.toString, 1)))
      }
    }
```

We expect the result to be:
```
2,1
3,1
```
But got
```
1,1
2,1
3,1
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #8916 from chenghao-intel/partition_filter.
2015-10-14 16:29:32 -07:00
Reynold Xin 2b5e31c7e9 [SPARK-11113] [SQL] Remove DeveloperApi annotation from private classes.
o.a.s.sql.catalyst and o.a.s.sql.execution are supposed to be private.

Author: Reynold Xin <rxin@databricks.com>

Closes #9121 from rxin/SPARK-11113.
2015-10-14 16:27:43 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Wenchen Fan 9a430a027f [SPARK-11068] [SQL] [FOLLOW-UP] move execution listener to util
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9119 from cloud-fan/callback.
2015-10-14 15:08:13 -07:00
Huaxin Gao 7e1308d37f [SPARK-8386] [SQL] add write.mode for insertIntoJDBC when the parm overwrite is false
the fix is for jira https://issues.apache.org/jira/browse/SPARK-8386

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #9042 from huaxingao/spark8386.
2015-10-14 12:31:29 -07:00
Yin Huai ce3f9a8065 [SPARK-11091] [SQL] Change spark.sql.canonicalizeView to spark.sql.nativeView.
https://issues.apache.org/jira/browse/SPARK-11091

Author: Yin Huai <yhuai@databricks.com>

Closes #9103 from yhuai/SPARK-11091.
2015-10-13 18:21:24 -07:00
Wenchen Fan 15ff85b316 [SPARK-11068] [SQL] add callback to query execution
With this feature, we can track the query plan, time cost, exception during query execution for spark users.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #9078 from cloud-fan/callback.
2015-10-13 17:59:32 -07:00
Wenchen Fan e170c22160 [SPARK-11032] [SQL] correctly handle having
We should not stop resolving having when the having condtion is resolved, or something like `count(1)` will crash.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #9105 from cloud-fan/having.
2015-10-13 17:11:22 -07:00
Michael Armbrust 328d1b3e4b [SPARK-11090] [SQL] Constructor for Product types from InternalRow
This is a first draft of the ability to construct expressions that will take a catalyst internal row and construct a Product (case class or tuple) that has fields with the correct names.  Support include:
 - Nested classes
 - Maps
 - Efficiently handling of arrays of primitive types

Not yet supported:
 - Case classes that require custom collection types (i.e. List instead of Seq).

Author: Michael Armbrust <michael@databricks.com>

Closes #9100 from marmbrus/productContructor.
2015-10-13 17:09:17 -07:00
Josh Rosen ef72673b23 [SPARK-11080] [SQL] Incorporate per-JVM id into ExprId to prevent unsafe cross-JVM comparisions
In the current implementation of named expressions' `ExprIds`, we rely on a per-JVM AtomicLong to ensure that expression ids are unique within a JVM. However, these expression ids will not be _globally_ unique. This opens the potential for id collisions if new expression ids happen to be created inside of tasks rather than on the driver.

There are currently a few cases where tasks allocate expression ids, which happen to be safe because those expressions are never compared to expressions created on the driver. In order to guard against the introduction of invalid comparisons between driver-created and executor-created expression ids, this patch extends `ExprId` to incorporate a UUID to identify the JVM that created the id, which prevents collisions.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9093 from JoshRosen/SPARK-11080.
2015-10-13 15:09:31 -07:00
Andrew Or b3ffac5178 [SPARK-10983] Unified memory manager
This patch unifies the memory management of the storage and execution regions such that either side can borrow memory from each other. When memory pressure arises, storage will be evicted in favor of execution. To avoid regressions in cases where storage is crucial, we dynamically allocate a fraction of space for storage that execution cannot evict. Several configurations are introduced:

- **spark.memory.fraction (default 0.75)**: ​fraction of the heap space used for execution and storage. The lower this is, the more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records.

- **spark.memory.storageFraction (default 0.5)**: size of the storage region within the space set aside by `s​park.memory.fraction`. ​Cached data may only be evicted if total storage exceeds this region.

- **spark.memory.useLegacyMode (default false)**: whether to use the memory management that existed in Spark 1.5 and before. This is mainly for backward compatibility.

For a detailed description of the design, see [SPARK-10000](https://issues.apache.org/jira/browse/SPARK-10000). This patch builds on top of the `MemoryManager` interface introduced in #9000.

Author: Andrew Or <andrew@databricks.com>

Closes #9084 from andrewor14/unified-memory-manager.
2015-10-13 13:49:59 -07:00
Sun Rui 5e3868ba13 [SPARK-10051] [SPARKR] Support collecting data of StructType in DataFrame
Two points in this PR:

1.    Originally thought was that a named R list is assumed to be a struct in SerDe. But this is problematic because some R functions will implicitly generate named lists that are not intended to be a struct when transferred by SerDe. So SerDe clients have to explicitly mark a names list as struct by changing its class from "list" to "struct".

2.    SerDe is in the Spark Core module, and data of StructType is represented as GenricRow which is defined in Spark SQL module. SerDe can't import GenricRow as in maven build  Spark SQL module depends on Spark Core module. So this PR adds a registration hook in SerDe to allow SQLUtils in Spark SQL module to register its functions for serialization and deserialization of StructType.

Author: Sun Rui <rui.sun@intel.com>

Closes #8794 from sun-rui/SPARK-10051.
2015-10-13 10:02:21 -07:00
Davies Liu d0cc79ccd0 [SPARK-11030] [SQL] share the SQLTab across sessions
The SQLTab will be shared by multiple sessions.

If we create multiple independent SQLContexts (not using newSession()), will still see multiple SQLTabs in the Spark UI.

Author: Davies Liu <davies@databricks.com>

Closes #9048 from davies/sqlui.
2015-10-13 09:57:53 -07:00
Davies Liu 6987c06793 [SPARK-11009] [SQL] fix wrong result of Window function in cluster mode
Currently, All windows function could generate wrong result in cluster sometimes.

The root cause is that AttributeReference is called in executor, then id of it may not be unique than others created in driver.

Here is the script that could reproduce the problem (run in local cluster):
```
from pyspark import SparkContext, HiveContext
from pyspark.sql.window import Window
from pyspark.sql.functions import rowNumber

sqlContext = HiveContext(SparkContext())
sqlContext.setConf("spark.sql.shuffle.partitions", "3")
df =  sqlContext.range(1<<20)
df2 = df.select((df.id % 1000).alias("A"), (df.id / 1000).alias('B'))
ws = Window.partitionBy(df2.A).orderBy(df2.B)
df3 = df2.select("client", "date", rowNumber().over(ws).alias("rn")).filter("rn < 0")
assert df3.count() == 0
```

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9050 from davies/wrong_window.
2015-10-13 09:43:33 -07:00
Davies Liu c4da5345a0 [SPARK-10990] [SPARK-11018] [SQL] improve unrolling of complex types
This PR improve the unrolling and read of complex types in columnar cache:
1) Using UnsafeProjection to do serialization of complex types, so they will not be serialized three times (two for actualSize)
2) Copy the bytes from UnsafeRow/UnsafeArrayData to ByteBuffer directly, avoiding the immediate byte[]
3) Using the underlying array in ByteBuffer to create UTF8String/UnsafeRow/UnsafeArrayData without copy.

Combine these optimizations,  we can reduce the unrolling time from 25s to 21s (20% less), reduce the scanning time from 3.5s to 2.5s (28% less).

```
df = sqlContext.read.parquet(path)
t = time.time()
df.cache()
df.count()
print 'unrolling', time.time() - t

for i in range(10):
    t = time.time()
    print df.select("*")._jdf.queryExecution().toRdd().count()
    print time.time() - t
```

The schema is
```
root
 |-- a: struct (nullable = true)
 |    |-- b: long (nullable = true)
 |    |-- c: string (nullable = true)
 |-- d: array (nullable = true)
 |    |-- element: long (containsNull = true)
 |-- e: map (nullable = true)
 |    |-- key: long
 |    |-- value: string (valueContainsNull = true)
```

Now the columnar cache depends on that UnsafeProjection support all the data types (including UDT), this PR also fix that.

Author: Davies Liu <davies@databricks.com>

Closes #9016 from davies/complex2.
2015-10-12 21:12:59 -07:00