Commit graph

14629 commits

Author SHA1 Message Date
Davies Liu 99a6e3c1e8 [SPARK-12951] [SQL] support spilling in generated aggregate
This PR add spilling support for generated TungstenAggregate.

If spilling happened, it's not that bad to do the iterator based sort-merge-aggregate (not generated).

The changes will be covered by TungstenAggregationQueryWithControlledFallbackSuite

Author: Davies Liu <davies@databricks.com>

Closes #10998 from davies/gen_spilling.
2016-02-02 19:47:44 -08:00
Adam Budde ff71261b65 [SPARK-13122] Fix race condition in MemoryStore.unrollSafely()
https://issues.apache.org/jira/browse/SPARK-13122

A race condition can occur in MemoryStore's unrollSafely() method if two threads that
return the same value for currentTaskAttemptId() execute this method concurrently. This
change makes the operation of reading the initial amount of unroll memory used, performing
the unroll, and updating the associated memory maps atomic in order to avoid this race
condition.

Initial proposed fix wraps all of unrollSafely() in a memoryManager.synchronized { } block. A cleaner approach might be introduce a mechanism that synchronizes based on task attempt ID. An alternative option might be to track unroll/pending unroll memory based on block ID rather than task attempt ID.

Author: Adam Budde <budde@amazon.com>

Closes #11012 from budde/master.
2016-02-02 19:35:33 -08:00
Nong Li 21112e8a14 [SPARK-12992] [SQL] Update parquet reader to support more types when decoding to ColumnarBatch.
This patch implements support for more types when doing the vectorized decode. There are
a few more types remaining but they should be very straightforward after this. This code
has a few copy and paste pieces but they are difficult to eliminate due to performance
considerations.

Specifically, this patch adds support for:
  - String, Long, Byte types
  - Dictionary encoding for those types.

Author: Nong Li <nong@databricks.com>

Closes #10908 from nongli/spark-12992.
2016-02-02 16:33:21 -08:00
Wenchen Fan 672032d0ab [SPARK-13020][SQL][TEST] fix random generator for map type
when we generate map, we first randomly pick a length, then create a seq of key value pair with the expected length, and finally call `toMap`. However, `toMap` will remove all duplicated keys, which makes the actual map size much less than we expected.

This PR fixes this problem by put keys in a set first, to guarantee we have enough keys to build a map with expected length.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10930 from cloud-fan/random-generator.
2016-02-03 08:26:35 +08:00
Davies Liu 6de6a97728 [SPARK-13150] [SQL] disable two flaky tests
Author: Davies Liu <davies@databricks.com>

Closes #11037 from davies/disable_flaky.
2016-02-02 16:24:31 -08:00
Kevin (Sangwoo) Kim b377b03531 [DOCS] Update StructType.scala
The example will throw error like
<console>:20: error: not found: value StructType

Need to add this line:
import org.apache.spark.sql.types._

Author: Kevin (Sangwoo) Kim <sangwookim.me@gmail.com>

Closes #10141 from swkimme/patch-1.
2016-02-02 13:24:21 -08:00
Gabriele Nizzoli d0df2ca409 [SPARK-13121][STREAMING] java mapWithState mishandles scala Option
Already merged into 1.6 branch, this PR is to commit to master the same change

Author: Gabriele Nizzoli <mail@nizzoli.net>

Closes #11028 from gabrielenizzoli/patch-1.
2016-02-02 13:20:01 -08:00
Davies Liu be5dd881f1 [SPARK-12913] [SQL] Improve performance of stat functions
As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one.

Author: Davies Liu <davies@databricks.com>

Closes #10960 from davies/stddev.
2016-02-02 11:50:14 -08:00
Reynold Xin 7f6e3ec79b [SPARK-13138][SQL] Add "logical" package prefix for ddl.scala
ddl.scala is defined in the execution package, and yet its reference of "UnaryNode" and "Command" are logical. This was fairly confusing when I was trying to understand the ddl code.

Author: Reynold Xin <rxin@databricks.com>

Closes #11021 from rxin/SPARK-13138.
2016-02-02 11:29:20 -08:00
Grzegorz Chilkiewicz b1835d7272 [SPARK-12711][ML] ML StopWordsRemover does not protect itself from column name duplication
Fixes problem and verifies fix by test suite.
Also - adds optional parameter: nullable (Boolean) to: SchemaUtils.appendColumn
and deduplicates SchemaUtils.appendColumn functions.

Author: Grzegorz Chilkiewicz <grzegorz.chilkiewicz@codilime.com>

Closes #10741 from grzegorz-chilkiewicz/master.
2016-02-02 11:16:24 -08:00
Daoyuan Wang 358300c795 [SPARK-13056][SQL] map column would throw NPE if value is null
Jira:
https://issues.apache.org/jira/browse/SPARK-13056

Create a map like
{ "a": "somestring", "b": null}
Query like
SELECT col["b"] FROM t1;
NPE would be thrown.

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

Closes #10964 from adrian-wang/npewriter.
2016-02-02 11:09:40 -08:00
Bryan Cutler cba1d6b659 [SPARK-12631][PYSPARK][DOC] PySpark clustering parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the clustering module.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #10610 from BryanCutler/param-desc-consistent-cluster-SPARK-12631.
2016-02-02 10:50:22 -08:00
hyukjinkwon b93830126c [SPARK-13114][SQL] Add a test for tokens more than the fields in schema
https://issues.apache.org/jira/browse/SPARK-13114

This PR adds a test for tokens more than the fields in schema.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11020 from HyukjinKwon/SPARK-13114.
2016-02-02 10:41:06 -08:00
Michael Armbrust 29d92181d0 [SPARK-13094][SQL] Add encoders for seq/array of primitives
Author: Michael Armbrust <michael@databricks.com>

Closes #11014 from marmbrus/seqEncoders.
2016-02-02 10:15:40 -08:00
Michael Armbrust 12a20c144f [SPARK-10820][SQL] Support for the continuous execution of structured queries
This is a follow up to 9aadcffabd that extends Spark SQL to allow users to _repeatedly_ optimize and execute structured queries.  A `ContinuousQuery` can be expressed using SQL, DataFrames or Datasets.  The purpose of this PR is only to add some initial infrastructure which will be extended in subsequent PRs.

## User-facing API

- `sqlContext.streamFrom` and `df.streamTo` return builder objects that are analogous to the `read/write` interfaces already available to executing queries in a batch-oriented fashion.
- `ContinuousQuery` provides an interface for interacting with a query that is currently executing in the background.

## Internal Interfaces
 - `StreamExecution` - executes streaming queries in micro-batches

The following are currently internal, but public APIs will be provided in a future release.
 - `Source` - an interface for providers of continually arriving data.  A source must have a notion of an `Offset` that monotonically tracks what data has arrived.  For fault tolerance, a source must be able to replay data given a start offset.
 - `Sink` - an interface that accepts the results of a continuously executing query.  Also responsible for tracking the offset that should be resumed from in the case of a failure.

## Testing
 - `MemoryStream` and `MemorySink` - simple implementations of source and sink that keep all data in memory and have methods for simulating durability failures
 - `StreamTest` - a framework for performing actions and checking invariants on a continuous query

Author: Michael Armbrust <michael@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Josh Rosen <rosenville@gmail.com>

Closes #11006 from marmbrus/structured-streaming.
2016-02-02 10:13:54 -08:00
Michael Armbrust 22ba21348b [SPARK-13087][SQL] Fix group by function for sort based aggregation
It is not valid to call `toAttribute` on a `NamedExpression` unless we know for sure that the child produced that `NamedExpression`.  The current code worked fine when the grouping expressions were simple, but when they were a derived value this blew up at execution time.

Author: Michael Armbrust <michael@databricks.com>

Closes #11013 from marmbrus/groupByFunction-master.
2016-02-02 16:48:59 +08:00
Reynold Xin b8666fd0e2 Closes #10662. Closes #10661 2016-02-01 23:37:06 -08:00
Reynold Xin 0fff5c6e63 [SPARK-13130][SQL] Make codegen variable names easier to read
1. Use lower case
2. Change long prefixes to something shorter (in this case I am changing only one: TungstenAggregate -> agg).

Author: Reynold Xin <rxin@databricks.com>

Closes #11017 from rxin/SPARK-13130.
2016-02-01 23:08:11 -08:00
felixcheung 0df3cfb8ab [SPARK-12790][CORE] Remove HistoryServer old multiple files format
Removed isLegacyLogDirectory code path and updated tests
andrewor14

Author: felixcheung <felixcheung_m@hotmail.com>

Closes #10860 from felixcheung/historyserverformat.
2016-02-01 16:55:21 -08:00
Sean Owen 715a19d56f [SPARK-12637][CORE] Print stage info of finished stages properly
Improve printing of StageInfo in onStageCompleted

See also https://github.com/apache/spark/pull/10585

Author: Sean Owen <sowen@cloudera.com>

Closes #10922 from srowen/SPARK-12637.
2016-02-01 16:23:17 -08:00
Reynold Xin be7a2fc071 [SPARK-13078][SQL] API and test cases for internal catalog
This pull request creates an internal catalog API. The creation of this API is the first step towards consolidating SQLContext and HiveContext. I envision we will have two different implementations in Spark 2.0: (1) a simple in-memory implementation, and (2) an implementation based on the current HiveClient (ClientWrapper).

I took a look at what Hive's internal metastore interface/implementation, and then created this API based on it. I believe this is the minimal set needed in order to achieve all the needed functionality.

Author: Reynold Xin <rxin@databricks.com>

Closes #10982 from rxin/SPARK-13078.
2016-02-01 14:11:52 -08:00
Jacek Laskowski a2973fed30 Fix for [SPARK-12854][SQL] Implement complex types support in Columna…
…rBatch

Fixes build for Scala 2.11.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #10946 from jaceklaskowski/SPARK-12854-fix.
2016-02-01 13:57:48 -08:00
Nong Li 064b029c6a [SPARK-13043][SQL] Implement remaining catalyst types in ColumnarBatch.
This includes: float, boolean, short, decimal and calendar interval.

Decimal is mapped to long or byte array depending on the size and calendar
interval is mapped to a struct of int and long.

The only remaining type is map. The schema mapping is straightforward but
we might want to revisit how we deal with this in the rest of the execution
engine.

Author: Nong Li <nong@databricks.com>

Closes #10961 from nongli/spark-13043.
2016-02-01 13:56:14 -08:00
Iulian Dragos c9b89a0a09 [SPARK-12979][MESOS] Don’t resolve paths on the local file system in Mesos scheduler
The driver filesystem is likely different from where the executors will run, so resolving paths (and symlinks, etc.) will lead to invalid paths on executors.

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #10923 from dragos/issue/canonical-paths.
2016-02-01 13:38:38 -08:00
Nilanjan Raychaudhuri a41b68b954 [SPARK-12265][MESOS] Spark calls System.exit inside driver instead of throwing exception
This takes over #10729 and makes sure that `spark-shell` fails with a proper error message. There is a slight behavioral change: before this change `spark-shell` would exit, while now the REPL is still there, but `sc` and `sqlContext` are not defined and the error is visible to the user.

Author: Nilanjan Raychaudhuri <nraychaudhuri@gmail.com>
Author: Iulian Dragos <jaguarul@gmail.com>

Closes #10921 from dragos/pr/10729.
2016-02-01 13:33:24 -08:00
Timothy Chen 51b03b71ff [SPARK-12463][SPARK-12464][SPARK-12465][SPARK-10647][MESOS] Fix zookeeper dir with mesos conf and add docs.
Fix zookeeper dir configuration used in cluster mode, and also add documentation around these settings.

Author: Timothy Chen <tnachen@gmail.com>

Closes #10057 from tnachen/fix_mesos_dir.
2016-02-01 12:45:02 -08:00
Lewuathe 711ce048a2 [ML][MINOR] Invalid MulticlassClassification reference in ml-guide
In [ml-guide](https://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation), there is invalid reference to `MulticlassClassificationEvaluator` apidoc.

https://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.ml.evaluation.MultiClassClassificationEvaluator

Author: Lewuathe <lewuathe@me.com>

Closes #10996 from Lewuathe/fix-typo-in-ml-guide.
2016-02-01 12:21:21 -08:00
Takeshi YAMAMURO da9146c91a [DOCS] Fix the jar location of datanucleus in sql-programming-guid.md
ISTM `lib` is better because `datanucleus` jars are located in `lib` for release builds.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #10901 from maropu/DocFix.
2016-02-01 12:02:06 -08:00
gatorsmile 8f26eb5ef6 [SPARK-12705][SPARK-10777][SQL] Analyzer Rule ResolveSortReferences
JIRA: https://issues.apache.org/jira/browse/SPARK-12705

**Scope:**
This PR is a general fix for sorting reference resolution when the child's `outputSet` does not have the order-by attributes (called, *missing attributes*):
  - UnaryNode support is limited to `Project`, `Window`, `Aggregate`, `Distinct`, `Filter`, `RepartitionByExpression`.
  - We will not try to resolve the missing references inside a subquery, unless the outputSet of this subquery contains it.

**General Reference Resolution Rules:**
  - Jump over the nodes with the following types: `Distinct`, `Filter`, `RepartitionByExpression`. Do not need to add missing attributes. The reason is their `outputSet` is decided by their `inputSet`, which is the `outputSet` of their children.
  - Group-by expressions in `Aggregate`: missing order-by attributes are not allowed to be added into group-by expressions since it will change the query result. Thus, in RDBMS, it is not allowed.
  - Aggregate expressions in `Aggregate`: if the group-by expressions in `Aggregate` contains the missing attributes but aggregate expressions do not have it, just add them into the aggregate expressions. This can resolve the analysisExceptions thrown by the three TCPDS queries.
  - `Project` and `Window` are special. We just need to add the missing attributes to their `projectList`.

**Implementation:**
  1. Traverse the whole tree in a pre-order manner to find all the resolvable missing order-by attributes.
  2. Traverse the whole tree in a post-order manner to add the found missing order-by attributes to the node if their `inputSet` contains the attributes.
  3. If the origins of the missing order-by attributes are different nodes, each pass only resolves the missing attributes that are from the same node.

**Risk:**
Low. This rule will be trigger iff ```!s.resolved && child.resolved``` is true. Thus, very few cases are affected.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10678 from gatorsmile/sortWindows.
2016-02-01 11:57:13 -08:00
gatorsmile 33c8a490f7 [SPARK-12989][SQL] Delaying Alias Cleanup after ExtractWindowExpressions
JIRA: https://issues.apache.org/jira/browse/SPARK-12989

In the rule `ExtractWindowExpressions`, we simply replace alias by the corresponding attribute. However, this will cause an issue exposed by the following case:

```scala
val data = Seq(("a", "b", "c", 3), ("c", "b", "a", 3)).toDF("A", "B", "C", "num")
  .withColumn("Data", struct("A", "B", "C"))
  .drop("A")
  .drop("B")
  .drop("C")

val winSpec = Window.partitionBy("Data.A", "Data.B").orderBy($"num".desc)
data.select($"*", max("num").over(winSpec) as "max").explain(true)
```
In this case, both `Data.A` and `Data.B` are `alias` in `WindowSpecDefinition`. If we replace these alias expression by their alias names, we are unable to know what they are since they will not be put in `missingExpr` too.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10963 from gatorsmile/seletStarAfterColDrop.
2016-02-01 11:22:02 -08:00
Shixiong Zhu 6075573a93 [SPARK-6847][CORE][STREAMING] Fix stack overflow issue when updateStateByKey is followed by a checkpointed dstream
Add a local property to indicate if checkpointing all RDDs that are marked with the checkpoint flag, and enable it in Streaming

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10934 from zsxwing/recursive-checkpoint.
2016-02-01 11:02:17 -08:00
Wenchen Fan c1da4d421a [SPARK-13093] [SQL] improve null check in nullSafeCodeGen for unary, binary and ternary expression
The current implementation is sub-optimal:

* If an expression is always nullable, e.g. `Unhex`, we can still remove null check for children if they are not nullable.
* If an expression has some non-nullable children, we can still remove null check for these children and keep null check for others.

This PR improves this by making the null check elimination more fine-grained.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10987 from cloud-fan/null-check.
2016-01-31 22:43:03 -08:00
Herman van Hovell 5a8b978fab [SPARK-13049] Add First/last with ignore nulls to functions.scala
This PR adds the ability to specify the ```ignoreNulls``` option to the functions dsl, e.g:
```df.select($"id", last($"value", ignoreNulls = true).over(Window.partitionBy($"id").orderBy($"other"))```

This PR is some where between a bug fix (see the JIRA) and a new feature. I am not sure if we should backport to 1.6.

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10957 from hvanhovell/SPARK-13049.
2016-01-31 13:56:13 -08:00
Liang-Chi Hsieh 0e6d92d042 [SPARK-12689][SQL] Migrate DDL parsing to the newly absorbed parser
JIRA: https://issues.apache.org/jira/browse/SPARK-12689

DDLParser processes three commands: createTable, describeTable and refreshTable.
This patch migrates the three commands to newly absorbed parser.

Author: Liang-Chi Hsieh <viirya@gmail.com>
Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #10723 from viirya/migrate-ddl-describe.
2016-01-30 23:05:29 -08:00
Cheng Lian a1303de0a0 [SPARK-13070][SQL] Better error message when Parquet schema merging fails
Make sure we throw better error messages when Parquet schema merging fails.

Author: Cheng Lian <lian@databricks.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #10979 from viirya/schema-merging-failure-message.
2016-01-30 23:02:49 -08:00
wangyang de28371998 [SPARK-13100][SQL] improving the performance of stringToDate method in DateTimeUtils.scala
In jdk1.7 TimeZone.getTimeZone() is synchronized, so use an instance variable to hold an GMT TimeZone object instead of instantiate it every time.

Author: wangyang <wangyang@haizhi.com>

Closes #10994 from wangyang1992/datetimeUtil.
2016-01-30 15:20:57 -08:00
Josh Rosen 289373b28c [SPARK-6363][BUILD] Make Scala 2.11 the default Scala version
This patch changes Spark's build to make Scala 2.11 the default Scala version. To be clear, this does not mean that Spark will stop supporting Scala 2.10: users will still be able to compile Spark for Scala 2.10 by following the instructions on the "Building Spark" page; however, it does mean that Scala 2.11 will be the default Scala version used by our CI builds (including pull request builds).

The Scala 2.11 compiler is faster than 2.10, so I think we'll be able to look forward to a slight speedup in our CI builds (it looks like it's about 2X faster for the Maven compile-only builds, for instance).

After this patch is merged, I'll update Jenkins to add new compile-only jobs to ensure that Scala 2.10 compilation doesn't break.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10608 from JoshRosen/SPARK-6363.
2016-01-30 00:20:28 -08:00
Wenchen Fan dab246f7e4 [SPARK-13098] [SQL] remove GenericInternalRowWithSchema
This class is only used for serialization of Python DataFrame. However, we don't require internal row there, so `GenericRowWithSchema` can also do the job.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10992 from cloud-fan/python.
2016-01-29 23:37:51 -08:00
Davies Liu e6a02c66d5 [SPARK-12914] [SQL] generate aggregation with grouping keys
This PR add support for grouping keys for generated TungstenAggregate.

Spilling and performance improvements for BytesToBytesMap will be done by followup PR.

Author: Davies Liu <davies@databricks.com>

Closes #10855 from davies/gen_keys.
2016-01-29 20:16:11 -08:00
Andrew Or 12252d1da9 [SPARK-13071] Coalescing HadoopRDD overwrites existing input metrics
This issue is causing tests to fail consistently in master with Hadoop 2.6 / 2.7. This is because for Hadoop 2.5+ we overwrite existing values of `InputMetrics#bytesRead` in each call to `HadoopRDD#compute`. In the case of coalesce, e.g.
```
sc.textFile(..., 4).coalesce(2).count()
```
we will call `compute` multiple times in the same task, overwriting `bytesRead` values from previous calls to `compute`.

For a regression test, see `InputOutputMetricsSuite.input metrics for old hadoop with coalesce`. I did not add a new regression test because it's impossible without significant refactoring; there's a lot of existing duplicate code in this corner of Spark.

This was caused by #10835.

Author: Andrew Or <andrew@databricks.com>

Closes #10973 from andrewor14/fix-input-metrics-coalesce.
2016-01-29 18:03:08 -08:00
Andrew Or 70e69fc4dd [SPARK-13088] Fix DAG viz in latest version of chrome
Apparently chrome removed `SVGElement.prototype.getTransformToElement`, which is used by our JS library dagre-d3 when creating edges. The real diff can be found here: 7d6c0002e4, which is taken from the fix in the main repo: 1ef067f1c6

Upstream issue: https://github.com/cpettitt/dagre-d3/issues/202

Author: Andrew Or <andrew@databricks.com>

Closes #10986 from andrewor14/fix-dag-viz.
2016-01-29 18:00:49 -08:00
Andrew Or e6ceac49a3 [SPARK-13096][TEST] Fix flaky verifyPeakExecutionMemorySet
Previously we would assert things before all events are guaranteed to have been processed. To fix this, just block until all events are actually processed, i.e. until the listener queue is empty.

https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/79/testReport/junit/org.apache.spark.util.collection/ExternalAppendOnlyMapSuite/spilling/

Author: Andrew Or <andrew@databricks.com>

Closes #10990 from andrewor14/accum-suite-less-flaky.
2016-01-29 17:59:41 -08:00
Reynold Xin 2cbc412821 [SPARK-13076][SQL] Rename ClientInterface -> HiveClient
And ClientWrapper -> HiveClientImpl.

I have some followup pull requests to introduce a new internal catalog, and I think this new naming reflects better the functionality of the two classes.

Author: Reynold Xin <rxin@databricks.com>

Closes #10981 from rxin/SPARK-13076.
2016-01-29 16:57:34 -08:00
Andrew Or e38b0baa38 [SPARK-13055] SQLHistoryListener throws ClassCastException
This is an existing issue uncovered recently by #10835. The reason for the exception was because the `SQLHistoryListener` gets all sorts of accumulators, not just the ones that represent SQL metrics. For example, the listener gets the `internal.metrics.shuffleRead.remoteBlocksFetched`, which is an Int, then it proceeds to cast the Int to a Long, which fails.

The fix is to mark accumulators representing SQL metrics using some internal metadata. Then we can identify which ones are SQL metrics and only process those in the `SQLHistoryListener`.

Author: Andrew Or <andrew@databricks.com>

Closes #10971 from andrewor14/fix-sql-history.
2016-01-29 13:45:03 -08:00
Cheng Lian 2b027e9a38 [SPARK-12818] Polishes spark-sketch module
Fixes various minor code and Javadoc styling issues.

Author: Cheng Lian <lian@databricks.com>

Closes #10985 from liancheng/sketch-polishing.
2016-01-29 12:01:13 -08:00
gatorsmile 5f686cc8b7 [SPARK-12656] [SQL] Implement Intersect with Left-semi Join
Our current Intersect physical operator simply delegates to RDD.intersect. We should remove the Intersect physical operator and simply transform a logical intersect into a semi-join with distinct. This way, we can take advantage of all the benefits of join implementations (e.g. managed memory, code generation, broadcast joins).

After a search, I found one of the mainstream RDBMS did the same. In their query explain, Intersect is replaced by Left-semi Join. Left-semi Join could help outer-join elimination in Optimizer, as shown in the PR: https://github.com/apache/spark/pull/10566

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10630 from gatorsmile/IntersectBySemiJoin.
2016-01-29 11:22:12 -08:00
Wenchen Fan c5f745ede0 [SPARK-13072] [SQL] simplify and improve murmur3 hash expression codegen
simplify(remove several unnecessary local variables) the generated code of hash expression, and avoid null check if possible.

generated code comparison for `hash(int, double, string, array<string>)`:
**before:**
```
  public UnsafeRow apply(InternalRow i) {
    /* hash(input[0, int],input[1, double],input[2, string],input[3, array<int>],42) */
    int value1 = 42;
    /* input[0, int] */
    int value3 = i.getInt(0);
    if (!false) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(value3, value1);
    }
    /* input[1, double] */
    double value5 = i.getDouble(1);
    if (!false) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(Double.doubleToLongBits(value5), value1);
    }
    /* input[2, string] */
    boolean isNull6 = i.isNullAt(2);
    UTF8String value7 = isNull6 ? null : (i.getUTF8String(2));
    if (!isNull6) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value7.getBaseObject(), value7.getBaseOffset(), value7.numBytes(), value1);
    }
    /* input[3, array<int>] */
    boolean isNull8 = i.isNullAt(3);
    ArrayData value9 = isNull8 ? null : (i.getArray(3));
    if (!isNull8) {
      int result10 = value1;
      for (int index11 = 0; index11 < value9.numElements(); index11++) {
        if (!value9.isNullAt(index11)) {
          final int element12 = value9.getInt(index11);
          result10 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(element12, result10);
        }
      }
      value1 = result10;
    }
  }
```
**after:**
```
  public UnsafeRow apply(InternalRow i) {
    /* hash(input[0, int],input[1, double],input[2, string],input[3, array<int>],42) */
    int value1 = 42;
    /* input[0, int] */
    int value3 = i.getInt(0);
    value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(value3, value1);
    /* input[1, double] */
    double value5 = i.getDouble(1);
    value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashLong(Double.doubleToLongBits(value5), value1);
    /* input[2, string] */
    boolean isNull6 = i.isNullAt(2);
    UTF8String value7 = isNull6 ? null : (i.getUTF8String(2));

    if (!isNull6) {
      value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value7.getBaseObject(), value7.getBaseOffset(), value7.numBytes(), value1);
    }

    /* input[3, array<int>] */
    boolean isNull8 = i.isNullAt(3);
    ArrayData value9 = isNull8 ? null : (i.getArray(3));
    if (!isNull8) {
      for (int index10 = 0; index10 < value9.numElements(); index10++) {
        final int element11 = value9.getInt(index10);
        value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashInt(element11, value1);
      }
    }

    rowWriter14.write(0, value1);
    return result12;
  }
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10974 from cloud-fan/codegen.
2016-01-29 10:24:23 -08:00
zhuol e4c1162b6b [SPARK-10873] Support column sort and search for History Server.
[SPARK-10873] Support column sort and search for History Server using jQuery DataTable and REST API. Before this commit, the history server was generated hard-coded html and can not support search, also, the sorting was disabled if there is any application that has more than one attempt. Supporting search and sort (over all applications rather than the 20 entries in the current page) in any case will greatly improve user experience.

1. Create the historypage-template.html for displaying application information in datables.
2. historypage.js uses jQuery to access the data from /api/v1/applications REST API, and use DataTable to display each application's information. For application that has more than one attempt, the RowsGroup is used to merge such entries while at the same time supporting sort and search.
3. "duration" and "lastUpdated" rest API are added to application's "attempts".
4. External javascirpt and css files for datatables, RowsGroup and jquery plugins are added with licenses clarified.

Snapshots for how it looks like now:

History page view:
![historypage](https://cloud.githubusercontent.com/assets/11683054/12184383/89bad774-b55a-11e5-84e4-b0276172976f.png)

Search:
![search](https://cloud.githubusercontent.com/assets/11683054/12184385/8d3b94b0-b55a-11e5-869a-cc0ef0a4242a.png)

Sort by started time:
![sort-by-started-time](https://cloud.githubusercontent.com/assets/11683054/12184387/8f757c3c-b55a-11e5-98c8-577936366566.png)

Author: zhuol <zhuol@yahoo-inc.com>

Closes #10648 from zhuoliu/10873.
2016-01-29 11:54:58 -06:00
Yanbo Liang e51b6eaa9e [SPARK-13032][ML][PYSPARK] PySpark support model export/import and take LinearRegression as example
* Implement ```MLWriter/MLWritable/MLReader/MLReadable``` for PySpark.
* Making ```LinearRegression``` to support ```save/load``` as example. After this merged, the work for other transformers/estimators will be easy, then we can list and distribute the tasks to the community.

cc mengxr jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #10469 from yanboliang/spark-11939.
2016-01-29 09:22:24 -08:00
Davies Liu 55561e7693 [SPARK-13031][SQL] cleanup codegen and improve test coverage
1. enable whole stage codegen during tests even there is only one operator supports that.
2. split doProduce() into two APIs: upstream() and doProduce()
3. generate prefix for fresh names of each operator
4. pass UnsafeRow to parent directly (avoid getters and create UnsafeRow again)
5. fix bugs and tests.

This PR re-open #10944 and fix the bug.

Author: Davies Liu <davies@databricks.com>

Closes #10977 from davies/gen_refactor.
2016-01-29 01:59:59 -08:00