aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-20343][BUILD] Force Avro 1.7.7 in sbt build to resolve build failure ↵hyukjinkwon2017-04-182-2/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | in SBT Hadoop 2.6 master on Jenkins ## What changes were proposed in this pull request? This PR proposes to force Avro's version to 1.7.7 in core to resolve the build failure as below: ``` [error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.6/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala:123: value createDatumWriter is not a member of org.apache.avro.generic.GenericData [error] writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema)) [error] ``` https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/2770/consoleFull Note that this is a hack and should be removed in the future. ## How was this patch tested? I only tested this actually overrides the dependency. I tried many ways but I was unable to reproduce this in my local. Sean also tried the way I did but he was also unable to reproduce this. Please refer the comments in https://github.com/apache/spark/pull/17477#issuecomment-294094092 Author: hyukjinkwon <gurwls223@gmail.com> Closes #17651 from HyukjinKwon/SPARK-20343-sbt.
* [SPARK-20344][SCHEDULER] Duplicate call in ↵Robert Stupp2017-04-181-16/+16
| | | | | | | | | | | | | | | | FairSchedulableBuilder.addTaskSetManager ## What changes were proposed in this pull request? Eliminate the duplicate call to `Pool.getSchedulableByName()` in `FairSchedulableBuilder.addTaskSetManager` ## How was this patch tested? ./dev/run-tests Author: Robert Stupp <snazy@snazy.de> Closes #17647 from snazy/20344-dup-call-master.
* [SPARK-17647][SQL][FOLLOWUP][MINOR] fix typoFelix Cheung2017-04-171-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? fix typo ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17663 from felixcheung/likedoctypo.
* [TEST][MINOR] Replace repartitionBy with distribute in CollapseRepartitionSuiteJacek Laskowski2017-04-171-11/+10
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replace non-existent `repartitionBy` with `distribute` in `CollapseRepartitionSuite`. ## How was this patch tested? local build and `catalyst/testOnly *CollapseRepartitionSuite` Author: Jacek Laskowski <jacek@japila.pl> Closes #17657 from jaceklaskowski/CollapseRepartitionSuite.
* Typo fix: distitrbuted -> distributedAndrew Ash2017-04-171-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Typo fix: distitrbuted -> distributed ## How was this patch tested? Existing tests Author: Andrew Ash <andrew@andrewash.com> Closes #17664 from ash211/patch-1.
* [SPARK-17647][SQL] Fix backslash escaping in 'LIKE' patterns.Jakob Odersky2017-04-174-87/+153
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping. A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead. --- Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features. | RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity | | --- | --- | --- | --- | | [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no | | [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes | | [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes | | [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no | | [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes | | [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes | | Current Spark | _, % | \ | yes | [1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket. The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed. 1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`. PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern. According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid. _Proposed new behaviour in Spark: throw AnalysisException_ 2. [x] Empty input, e.g. `'' like ''` Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule. 3. [x] Escape before a non-special character, e.g. `'a' like '\a'`. Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same. According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign. _Proposed new behaviour in Spark: throw AnalysisException_ The current specification is also described in the operator's source code in this patch. ## How was this patch tested? Extra case in regex unit tests. Author: Jakob Odersky <jakob@odersky.com> This patch had conflicts when merged, resolved by Committer: Reynold Xin <rxin@databricks.com> Closes #15398 from jodersky/SPARK-17647.
* [SPARK-20349][SQL] ListFunctions returns duplicate functions after using ↵Xiao Li2017-04-173-8/+34
| | | | | | | | | | | | | | | | persistent functions ### What changes were proposed in this pull request? The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it. It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names. ### How was this patch tested? Added test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17646 from gatorsmile/showFunctions.
* [SPARK-19828][R][FOLLOWUP] Rename asJsonArray to as.json.array in from_json ↵hyukjinkwon2017-04-172-5/+5
| | | | | | | | | | | | | | | | | | | | function in R ## What changes were proposed in this pull request? This was suggested to be `as.json.array` at the first place in the PR to SPARK-19828 but we could not do this as the lint check emits an error for multiple dots in the variable names. After SPARK-20278, now we are able to use `multiple.dots.in.names`. `asJsonArray` in `from_json` function is still able to be changed as 2.2 is not released yet. So, this PR proposes to rename `asJsonArray` to `as.json.array`. ## How was this patch tested? Jenkins tests, local tests with `./R/run-tests.sh` and manual `./dev/lint-r`. Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17653 from HyukjinKwon/SPARK-19828-followup.
* [SPARK-20278][R] Disable 'multiple_dots_linter' lint rule that is against ↵hyukjinkwon2017-04-161-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | project's code style ## What changes were proposed in this pull request? Currently, multi-dot separated variables in R is not allowed. For example, ```diff setMethod("from_json", signature(x = "Column", schema = "structType"), - function(x, schema, asJsonArray = FALSE, ...) { + function(x, schema, as.json.array = FALSE, ...) { if (asJsonArray) { jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", "createArrayType", ``` produces an error as below: ``` R/functions.R:2462:31: style: Words within variable and function names should be separated by '_' rather than '.'. function(x, schema, as.json.array = FALSE, ...) { ^~~~~~~~~~~~~ ``` This seems against https://google.github.io/styleguide/Rguide.xml#identifiers which says > The preferred form for variable names is all lower case letters and words separated with dots This looks because lintr by default https://github.com/jimhester/lintr follows http://r-pkgs.had.co.nz/style.html as written in the README.md. Few cases seems not following Google's one as "a few tweaks". Per [SPARK-6813](https://issues.apache.org/jira/browse/SPARK-6813), we follow Google's R Style Guide with few exceptions https://google.github.io/styleguide/Rguide.xml. This is also merged into Spark's website - https://github.com/apache/spark-website/pull/43 Also, it looks we have no limit on function name. This rule also looks affecting to the name of functions as written in the README.md. > `multiple_dots_linter`: check that function and variable names are separated by _ rather than .. ## How was this patch tested? Manually tested `./dev/lint-r`with the manual change below in `R/functions.R`: ```diff setMethod("from_json", signature(x = "Column", schema = "structType"), - function(x, schema, asJsonArray = FALSE, ...) { + function(x, schema, as.json.array = FALSE, ...) { if (asJsonArray) { jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", "createArrayType", ``` **Before** ```R R/functions.R:2462:31: style: Words within variable and function names should be separated by '_' rather than '.'. function(x, schema, as.json.array = FALSE, ...) { ^~~~~~~~~~~~~ ``` **After** ``` lintr checks passed. ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #17590 from HyukjinKwon/disable-dot-in-name.
* [SPARK-20343][BUILD] Add avro dependency in core POM to resolve build ↵hyukjinkwon2017-04-161-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failure in SBT Hadoop 2.6 master on Jenkins ## What changes were proposed in this pull request? This PR proposes to add ``` <dependency> <groupId>org.apache.avro</groupId> <artifactId>avro</artifactId> </dependency> ``` in core POM to see if it resolves the build failure as below: ``` [error] /home/jenkins/workspace/spark-master-test-sbt-hadoop-2.6/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala:123: value createDatumWriter is not a member of org.apache.avro.generic.GenericData [error] writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema)) [error] ``` https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/2770/consoleFull ## How was this patch tested? I tried many ways but I was unable to reproduce this in my local. Sean also tried the way I did but he was also unable to reproduce this. Please refer the comments in https://github.com/apache/spark/pull/17477#issuecomment-294094092 Author: hyukjinkwon <gurwls223@gmail.com> Closes #17642 from HyukjinKwon/SPARK-20343.
* [SPARK-19740][MESOS] Add support in Spark to pass arbitrary parameters into ↵Ji Yan2017-04-163-3/+96
| | | | | | | | | | | | | | | | docker when running on mesos with docker containerizer ## What changes were proposed in this pull request? Allow passing in arbitrary parameters into docker when launching spark executors on mesos with docker containerizer tnachen ## How was this patch tested? Manually built and tested with passed in parameter Author: Ji Yan <jiyan@Jis-MacBook-Air.local> Closes #17109 from yanji84/ji/allow_set_docker_user.
* [SPARK-20335][SQL] Children expressions of Hive UDF impacts the determinism ↵Xiao Li2017-04-164-3/+47
| | | | | | | | | | | | | | | | | | | | | | | | | | of Hive UDF ### What changes were proposed in this pull request? ```JAVA /** * Certain optimizations should not be applied if UDF is not deterministic. * Deterministic UDF returns same result each time it is invoked with a * particular input. This determinism just needs to hold within the context of * a query. * * return true if the UDF is deterministic */ boolean deterministic() default true; ``` Based on the definition of [UDFType](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java#L42-L50), when Hive UDF's children are non-deterministic, Hive UDF is also non-deterministic. ### How was this patch tested? Added test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17635 from gatorsmile/udfDeterministic.
* [SPARK-19716][SQL][FOLLOW-UP] UnresolvedMapObjects should always be serializableWenchen Fan2017-04-161-25/+31
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable. Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #17639 from cloud-fan/minor.
* [SPARK-20316][SQL] Val and Var should strictly follow the Scala syntaxouyangxiaochen2017-04-151-2/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? val and var should strictly follow the Scala syntax ## How was this patch tested? manual test and exisiting test cases Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn> Closes #17628 from ouyangxiaochen/spark-413.
* [SPARK-20318][SQL] Use Catalyst type for min/max in ColumnStat for ease of ↵wangzhenhua2017-04-1410-182/+189
| | | | | | | | | | | | | | | | | | | | | estimation ## What changes were proposed in this pull request? Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation. To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore. This pr also fixes a bug for boolean type in `IN` condition. ## How was this patch tested? The changes for ColumnStat are covered by existing tests. For bug fix, a new test for boolean type in IN condition is added Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17630 from wzhfy/refactorColumnStat.
* [SPARK-20038][SQL] FileFormatWriter.ExecuteWriteTask.releaseResources() ↵Steve Loughran2017-04-131-4/+10
| | | | | | | | | | | | | | | | | | implementations to be re-entrant ## What changes were proposed in this pull request? have the`FileFormatWriter.ExecuteWriteTask.releaseResources()` implementations set `currentWriter=null` in a finally clause. This guarantees that if the first call to `currentWriter()` throws an exception, the second releaseResources() call made during the task cancel process will not trigger a second attempt to close the stream. ## How was this patch tested? Tricky. I've been fixing the underlying cause when I saw the problem [HADOOP-14204](https://issues.apache.org/jira/browse/HADOOP-14204), but SPARK-10109 shows I'm not the first to have seen this. I can't replicate it locally any more, my code no longer being broken. code review, however, should be straightforward Author: Steve Loughran <stevel@hortonworks.com> Closes #17364 from steveloughran/stevel/SPARK-20038-close.
* [SPARK-20232][PYTHON] Improve combineByKey docsDavid Gingrich2017-04-131-5/+19
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Improve combineByKey documentation: * Add note on memory allocation * Change example code to use different mergeValue and mergeCombiners ## How was this patch tested? Doctest. ## Legal This is my original work and I license the work to the project under the project’s open source license. Author: David Gingrich <david@textio.com> Closes #17545 from dgingrich/topic-spark-20232-combinebykey-docs.
* [SPARK-20233][SQL] Apply star-join filter heuristics to dynamic programming ↵Ioana Delaney2017-04-134-9/+571
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | join enumeration ## What changes were proposed in this pull request? Implements star-join filter to reduce the search space for dynamic programming join enumeration. Consider the following join graph: ``` T1 D1 - T2 - T3 \ / F1 | D2 star-join: {F1, D1, D2} non-star: {T1, T2, T3} ``` The following join combinations will be generated: ``` level 0: (F1), (D1), (D2), (T1), (T2), (T3) level 1: {F1, D1}, {F1, D2}, {T2, T3} level 2: {F1, D1, D2} level 3: {F1, D1, D2, T1}, {F1, D1, D2, T2} level 4: {F1, D1, D2, T1, T2}, {F1, D1, D2, T2, T3 } level 6: {F1, D1, D2, T1, T2, T3} ``` ## How was this patch tested? New test suite ```StarJOinCostBasedReorderSuite.scala```. Author: Ioana Delaney <ioanamdelaney@gmail.com> Closes #17546 from ioana-delaney/starSchemaCBOv3.
* [SPARK-20284][CORE] Make {Des,S}erializationStream extend CloseableSergei Lebedev2017-04-131-4/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR allows to use `SerializationStream` and `DeserializationStream` in try-with-resources. ## How was this patch tested? `core` unit tests. Author: Sergei Lebedev <s.lebedev@criteo.com> Closes #17598 from superbobry/compression-stream-closeable.
* [SPARK-20265][MLLIB] Improve Prefix'span pre-processing efficiencySyrux2017-04-132-35/+115
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Improve PrefixSpan pre-processing efficency by preventing sequences of zero in the cleaned database. The efficiency gain is reflected in the following graph : https://postimg.org/image/9x6ireuvn/ ## How was this patch tested? Using MLlib's PrefixSpan existing tests and tests of my own on the 8 datasets shown in the graph. All result obtained were stricly the same as the original implementation (without this change). dev/run-tests was also runned, no error were found. Author : Cyril de Vogelaere <cyril.devogelaeregmail.com> Author: Syrux <pokcyril@hotmail.com> Closes #17575 from Syrux/SPARK-20265.
* [SPARK-20189][DSTREAM] Fix spark kinesis testcases to remove deprecated ↵Yash Sharma2017-04-132-22/+38
| | | | | | | | | | | | | | | | | | | createStream and use Builders ## What changes were proposed in this pull request? The spark-kinesis testcases use the KinesisUtils.createStream which are deprecated now. Modify the testcases to use the recommended KinesisInputDStream.builder instead. This change will also enable the testcases to automatically use the session tokens automatically. ## How was this patch tested? All the existing testcases work fine as expected with the changes. https://issues.apache.org/jira/browse/SPARK-20189 Author: Yash Sharma <ysharma@atlassian.com> Closes #17506 from yssharma/ysharma/cleanup_kinesis_testcases.
* [SPARK-20131][CORE] Don't use `this` lock in StandaloneSchedulerBackend.stopShixiong Zhu2017-04-121-16/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `o.a.s.streaming.StreamingContextSuite.SPARK-18560 Receiver data should be deserialized properly` is flaky is because there is a potential dead-lock in StandaloneSchedulerBackend which causes `await` timeout. Here is the related stack trace: ``` "Thread-31" #211 daemon prio=5 os_prio=31 tid=0x00007fedd4808000 nid=0x16403 waiting on condition [0x00007000239b7000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x000000079b49ca10> (a scala.concurrent.impl.Promise$CompletionLatch) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:208) at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:218) at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223) at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:201) at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:92) at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:76) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.stop(CoarseGrainedSchedulerBackend.scala:402) at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.org$apache$spark$scheduler$cluster$StandaloneSchedulerBackend$$stop(StandaloneSchedulerBackend.scala:213) - locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.stop(StandaloneSchedulerBackend.scala:116) - locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.TaskSchedulerImpl.stop(TaskSchedulerImpl.scala:517) at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1657) at org.apache.spark.SparkContext$$anonfun$stop$8.apply$mcV$sp(SparkContext.scala:1921) at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1302) at org.apache.spark.SparkContext.stop(SparkContext.scala:1920) at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:708) at org.apache.spark.streaming.StreamingContextSuite$$anonfun$43$$anonfun$apply$mcV$sp$66$$anon$3.run(StreamingContextSuite.scala:827) "dispatcher-event-loop-3" #18 daemon prio=5 os_prio=31 tid=0x00007fedd603a000 nid=0x6203 waiting for monitor entry [0x0000700003be4000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint.org$apache$spark$scheduler$cluster$CoarseGrainedSchedulerBackend$DriverEndpoint$$makeOffers(CoarseGrainedSchedulerBackend.scala:253) - waiting to lock <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend) at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anonfun$receive$1.applyOrElse(CoarseGrainedSchedulerBackend.scala:124) at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117) at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205) at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101) at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:213) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` This PR removes `synchronized` and changes `stopping` to AtomicBoolean to ensure idempotent to fix the dead-lock. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17610 from zsxwing/SPARK-20131.
* [SPARK-15354][FLAKY-TEST] TopologyAwareBlockReplicationPolicyBehavior.Peers ↵Wenchen Fan2017-04-131-2/+11
| | | | | | | | | | | | | | | | | | in 2 racks ## What changes were proposed in this pull request? `TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks` is failing occasionally: https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.storage.TopologyAwareBlockReplicationPolicyBehavior&test_name=Peers+in+2+racks. This is because, when we generate 10 block manager id to test, they may all belong to the same rack, as the rack is randomly picked. This PR fixes this problem by forcing each rack to be picked at least once. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #17624 from cloud-fan/test.
* [SPARK-20301][FLAKY-TEST] Fix Hadoop Shell.runCommand flakiness in ↵Burak Yavuz2017-04-122-30/+32
| | | | | | | | | | | | | | | | | | | | | | Structured Streaming tests ## What changes were proposed in this pull request? Some Structured Streaming tests show flakiness such as: ``` [info] - prune results by current_date, complete mode - 696 *** FAILED *** (10 seconds, 937 milliseconds) [info] Timed out while stopping and waiting for microbatchthread to terminate.: The code passed to failAfter did not complete within 10 seconds. ``` This happens when we wait for the stream to stop, but it doesn't. The reason it doesn't stop is that we interrupt the microBatchThread, but Hadoop's `Shell.runCommand` swallows the interrupt exception, and the exception is not propagated upstream to the microBatchThread. Then this thread continues to run, only to start blocking on the `streamManualClock`. ## How was this patch tested? Thousand retries locally and [Jenkins](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75720/testReport) of the flaky tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #17613 from brkyvz/flaky-stream-agg.
* [SPARK-19570][PYSPARK] Allow to disable hive in pyspark shellJeff Zhang2017-04-121-6/+16
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-15236 do this for scala shell, this ticket is for pyspark shell. This is not only for pyspark itself, but can also benefit downstream project like livy which use shell.py for its interactive session. For now, livy has no control of whether enable hive or not. ## How was this patch tested? I didn't find a way to add test for it. Just manually test it. Run `bin/pyspark --master local --conf spark.sql.catalogImplementation=in-memory` and verify hive is not enabled. Author: Jeff Zhang <zjffdu@apache.org> Closes #16906 from zjffdu/SPARK-19570.
* [SPARK-20304][SQL] AssertNotNull should not include path in string ↵Reynold Xin2017-04-121-0/+2
| | | | | | | | | | | | | | representation ## What changes were proposed in this pull request? AssertNotNull's toString/simpleString dumps the entire walkedTypePath. walkedTypePath is used for error message reporting and shouldn't be part of the output. ## How was this patch tested? Manually tested. Author: Reynold Xin <rxin@databricks.com> Closes #17616 from rxin/SPARK-20304.
* [SPARK-20303][SQL] Rename createTempFunction to registerFunctionXiao Li2017-04-127-63/+53
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere. In the next PRs, the remaining Function-related APIs also need cleanups. ### How was this patch tested? Existing test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17615 from gatorsmile/cleanupCreateTempFunction.
* [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkinshyukjinkwon2017-04-1249-106/+140
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable. There are several problems with it: - It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?". - > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up. (see joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627)) To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above. There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013 Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings. ## How was this patch tested? Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`. This was tested via manually adding `time.time()` as below: ```diff profiles_and_goals = build_profiles + sbt_goals print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ", " ".join(profiles_and_goals)) + import time + st = time.time() exec_sbt(profiles_and_goals) + print("Elapsed :[%s]" % str(time.time() - st)) ``` produces ``` ... ======================================================================== Building Unidoc API Documentation ======================================================================== ... [info] Main Java API documentation successful. ... Elapsed :[94.8746569157] ... Author: hyukjinkwon <gurwls223@gmail.com> Closes #17477 from HyukjinKwon/SPARK-18692.
* [SPARK-20296][TRIVIAL][DOCS] Count distinct error message for streamingjtoka2017-04-121-3/+2
| | | | | | | | | ## What changes were proposed in this pull request? Update count distinct error message for streaming datasets/dataframes to match current behavior. These aggregations are not yet supported, regardless of whether the dataset/dataframe is aggregated. Author: jtoka <jason.tokayer@gmail.com> Closes #17609 from jtoka/master.
* [SPARK-20302][SQL] Short circuit cast when from and to types are ↵Reynold Xin2017-04-124-23/+113
| | | | | | | | | | | | | | structurally the same ## What changes were proposed in this pull request? When we perform a cast expression and the from and to types are structurally the same (having the same structure but different field names), we should be able to skip the actual cast. ## How was this patch tested? Added unit tests for the newly introduced functions. Author: Reynold Xin <rxin@databricks.com> Closes #17614 from rxin/SPARK-20302.
* [SPARK-20298][SPARKR][MINOR] fixed spelling mistake "charactor"Brendan Dwyer2017-04-123-9/+9
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixed spelling of "charactor" ## How was this patch tested? Spelling change only Author: Brendan Dwyer <brendan.dwyer@ibm.com> Closes #17611 from bdwyer2/SPARK-20298.
* [MINOR][DOCS] JSON APIs related documentation fixeshyukjinkwon2017-04-126-11/+13
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes corrections related to JSON APIs as below: - Rendering links in Python documentation - Replacing `RDD` to `Dataset` in programing guide - Adding missing description about JSON Lines consistently in `DataFrameReader.json` in Python API - De-duplicating little bit of `DataFrameReader.json` in Scala/Java API ## How was this patch tested? Manually build the documentation via `jekyll build`. Corresponding snapstops will be left on the codes. Note that currently there are Javadoc8 breaks in several places. These are proposed to be handled in https://github.com/apache/spark/pull/17477. So, this PR does not fix those. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17602 from HyukjinKwon/minor-json-documentation.
* [MINOR][DOCS] Fix spacings in Structured Streaming Programming GuideLee Dongjin2017-04-121-2/+2
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. Omitted space between the sentences: `... on static data.The Spark SQL engine will ...` -> `... on static data. The Spark SQL engine will ...` 2. Omitted colon in Output Model section. ## How was this patch tested? None. Author: Lee Dongjin <dongjin@apache.org> Closes #17564 from dongjinleekr/feature/fix-programming-guide.
* [SPARK-19993][SQL] Caching logical plans containing subquery expressions ↵Dilip Biswal2017-04-125-26/+198
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | does not work. ## What changes were proposed in this pull request? The sameResult() method does not work when the logical plan contains subquery expressions. **Before the fix** ```SQL scala> val ds = spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)") ds: org.apache.spark.sql.DataFrame = [c1: int] scala> ds.cache res13: ds.type = [c1: int] scala> spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)").explain(true) == Analyzed Logical Plan == c1: int Project [c1#86] +- Filter c1#86 IN (list#78 [c1#86]) : +- Project [c1#87] : +- Filter (outer(c1#86) = c1#87) : +- SubqueryAlias s2 : +- Relation[c1#87] parquet +- SubqueryAlias s1 +- Relation[c1#86] parquet == Optimized Logical Plan == Join LeftSemi, ((c1#86 = c1#87) && (c1#86 = c1#87)) :- Relation[c1#86] parquet +- Relation[c1#87] parquet ``` **Plan after fix** ```SQL == Analyzed Logical Plan == c1: int Project [c1#22] +- Filter c1#22 IN (list#14 [c1#22]) : +- Project [c1#23] : +- Filter (outer(c1#22) = c1#23) : +- SubqueryAlias s2 : +- Relation[c1#23] parquet +- SubqueryAlias s1 +- Relation[c1#22] parquet == Optimized Logical Plan == InMemoryRelation [c1#22], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) +- *BroadcastHashJoin [c1#1, c1#1], [c1#2, c1#2], LeftSemi, BuildRight :- *FileScan parquet default.s1[c1#1] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int> +- BroadcastExchange HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295)))) +- *FileScan parquet default.s2[c1#2] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s2], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int> ``` ## How was this patch tested? New tests are added to CachedTableSuite. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #17330 from dilipbiswal/subquery_cache_final.
* [SPARK-20291][SQL] NaNvl(FloatType, NullType) should not be cast to ↵DB Tsai2017-04-123-6/+12
| | | | | | | | | | | | | | | | | | | | | | NaNvl(DoubleType, DoubleType) ## What changes were proposed in this pull request? `NaNvl(float value, null)` will be converted into `NaNvl(float value, Cast(null, DoubleType))` and finally `NaNvl(Cast(float value, DoubleType), Cast(null, DoubleType))`. This will cause mismatching in the output type when the input type is float. By adding extra rule in TypeCoercion can resolve this issue. ## How was this patch tested? unite tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: DB Tsai <dbt@netflix.com> Closes #17606 from dbtsai/fixNaNvl.
* [MINOR][DOCS] Update supported versions for Hive MetastoreDongjoon Hyun2017-04-111-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Since SPARK-18112 and SPARK-13446, Apache Spark starts to support reading Hive metastore 2.0 ~ 2.1.1. This updates the docs. ## How was this patch tested? N/A Author: Dongjoon Hyun <dongjoon@apache.org> Closes #17612 from dongjoon-hyun/metastore.
* [SPARK-19505][PYTHON] AttributeError on Exception.message in Python3David Gingrich2017-04-114-5/+54
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Added `util._message_exception` helper to use `str(e)` when `e.message` is unavailable (Python3). Grepped for all occurrences of `.message` in `pyspark/` and these were the only occurrences. ## How was this patch tested? - Doctests for helper function ## Legal This is my original work and I license the work to the project under the project’s open source license. Author: David Gingrich <david@textio.com> Closes #16845 from dgingrich/topic-spark-19505-py3-exceptions.
* [SPARK-20289][SQL] Use StaticInvoke to box primitive typesReynold Xin2017-04-112-21/+20
| | | | | | | | | | | | ## What changes were proposed in this pull request? Dataset typed API currently uses NewInstance to box primitive types (i.e. calling the constructor). Instead, it'd be slightly more idiomatic in Java to use PrimitiveType.valueOf, which can be invoked using StaticInvoke expression. ## How was this patch tested? The change should be covered by existing tests for Dataset encoders. Author: Reynold Xin <rxin@databricks.com> Closes #17604 from rxin/SPARK-20289.
* [SPARK-20175][SQL] Exists should not be evaluated in Join operatorLiang-Chi Hsieh2017-04-112-1/+12
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Similar to `ListQuery`, `Exists` should not be evaluated in `Join` operator too. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17491 from viirya/dont-push-exists-to-join.
* [SPARK-20274][SQL] support compatible array element type in encoderWenchen Fan2017-04-113-9/+40
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a regression caused by SPARK-19716. Before SPARK-19716, we will cast an array field to the expected array type. However, after SPARK-19716, the cast is removed, but we forgot to push the cast to the element level. ## How was this patch tested? new regression tests Author: Wenchen Fan <wenchen@databricks.com> Closes #17587 from cloud-fan/array.
* Document Master URL format in high availability set upMirrorZ2017-04-111-0/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add documentation for adding master url in multi host, port format for standalone cluster with high availability with zookeeper. Referring documentation [Standby Masters with ZooKeeper](http://spark.apache.org/docs/latest/spark-standalone.html#standby-masters-with-zookeeper) ## How was this patch tested? Documenting the functionality already present. Author: MirrorZ <chandrika3437@gmail.com> Closes #17584 from MirrorZ/master.
* [SPARK-20097][ML] Fix visibility discrepancy with numInstances and ↵Benjamin Fradet2017-04-112-2/+4
| | | | | | | | | | | | | | | | | degreesOfFreedom in LR and GLR ## What changes were proposed in this pull request? - made `numInstances` public in GLR - made `degreesOfFreedom` public in LR ## How was this patch tested? reran the concerned test suites Author: Benjamin Fradet <benjamin.fradet@gmail.com> Closes #17431 from BenFradet/SPARK-20097.
* [SPARK-17564][TESTS] Fix flaky ↵Shixiong Zhu2017-04-101-0/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | RequestTimeoutIntegrationSuite.furtherRequestsDelay ## What changes were proposed in this pull request? This PR fixs the following failure: ``` sbt.ForkMain$ForkError: java.lang.AssertionError: null at org.junit.Assert.fail(Assert.java:86) at org.junit.Assert.assertTrue(Assert.java:41) at org.junit.Assert.assertTrue(Assert.java:52) at org.apache.spark.network.RequestTimeoutIntegrationSuite.furtherRequestsDelay(RequestTimeoutIntegrationSuite.java:230) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at org.junit.runner.JUnitCore.run(JUnitCore.java:115) at com.novocode.junit.JUnitRunner$1.execute(JUnitRunner.java:132) at sbt.ForkMain$Run$2.call(ForkMain.java:296) at sbt.ForkMain$Run$2.call(ForkMain.java:286) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` It happens several times per month on [Jenkins](http://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.network.RequestTimeoutIntegrationSuite&test_name=furtherRequestsDelay). The failure is because `callback1` may not be called before `assertTrue(callback1.failure instanceof IOException);`. It's pretty easy to reproduce this error by adding a sleep before this line: https://github.com/apache/spark/blob/379b0b0bbdbba2278ce3bcf471bd75f6ffd9cf0d/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java#L267 The fix is straightforward: just use the latch to wait until `callback1` is called. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17599 from zsxwing/SPARK-17564.
* [SPARK-20283][SQL] Add preOptimizationBatchesReynold Xin2017-04-101-1/+7
| | | | | | | | | | | | ## What changes were proposed in this pull request? We currently have postHocOptimizationBatches, but not preOptimizationBatches. This patch adds preOptimizationBatches so the optimizer debugging extensions are symmetric. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #17595 from rxin/SPARK-20283.
* [SPARK-20282][SS][TESTS] Write the commit log first to fix a race contion in ↵Shixiong Zhu2017-04-101-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | tests ## What changes were proposed in this pull request? This PR fixes the following failure: ``` sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: Assert on query failed: == Progress == AssertOnQuery(<condition>, ) StopStream AddData to MemoryStream[value#30891]: 1,2 StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock35cdc93a,Map()) CheckAnswer: [6],[3] StopStream => AssertOnQuery(<condition>, ) AssertOnQuery(<condition>, ) StartStream(OneTimeTrigger,org.apache.spark.util.SystemClockcdb247d,Map()) CheckAnswer: [6],[3] StopStream AddData to MemoryStream[value#30891]: 3 StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock55394e4d,Map()) CheckLastBatch: [2] StopStream AddData to MemoryStream[value#30891]: 0 StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock749aa997,Map()) ExpectFailure[org.apache.spark.SparkException, isFatalError: false] AssertOnQuery(<condition>, ) AssertOnQuery(<condition>, incorrect start offset or end offset on exception) == Stream == Output Mode: Append Stream state: not started Thread state: dead == Sink == 0: [6] [3] == Plan == at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) at org.scalatest.Assertions$class.fail(Assertions.scala:1328) at org.scalatest.FunSuite.fail(FunSuite.scala:1555) at org.apache.spark.sql.streaming.StreamTest$class.failTest$1(StreamTest.scala:347) at org.apache.spark.sql.streaming.StreamTest$class.verify$1(StreamTest.scala:318) at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:483) at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:357) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.sql.streaming.StreamTest$class.liftedTree1$1(StreamTest.scala:357) at org.apache.spark.sql.streaming.StreamTest$class.testStream(StreamTest.scala:356) at org.apache.spark.sql.streaming.StreamingQuerySuite.testStream(StreamingQuerySuite.scala:41) at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply$mcV$sp(StreamingQuerySuite.scala:166) at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161) at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161) at org.apache.spark.sql.catalyst.util.package$.quietly(package.scala:42) at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply$mcV$sp(SQLTestUtils.scala:268) at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268) at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfterEach$$super$runTest(StreamingQuerySuite.scala:41) at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255) at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingQuerySuite.scala:41) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at org.apache.spark.sql.streaming.StreamingQuerySuite.runTest(StreamingQuerySuite.scala:41) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:381) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:31) at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$run(StreamingQuerySuite.scala:41) at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) at org.apache.spark.sql.streaming.StreamingQuerySuite.run(StreamingQuerySuite.scala:41) at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:357) at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:502) at sbt.ForkMain$Run$2.call(ForkMain.java:296) at sbt.ForkMain$Run$2.call(ForkMain.java:286) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` The failure is because `CheckAnswer` will run once `committedOffsets` is updated. Then writing the commit log may be interrupted by the following `StopStream`. This PR just change the order to write the commit log first. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17594 from zsxwing/SPARK-20282.
* [SPARK-20285][TESTS] Increase the pyspark streaming test timeout to 30 secondsShixiong Zhu2017-04-101-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Saw the following failure locally: ``` Traceback (most recent call last): File "/home/jenkins/workspace/python/pyspark/streaming/tests.py", line 351, in test_cogroup self._test_func(input, func, expected, sort=True, input2=input2) File "/home/jenkins/workspace/python/pyspark/streaming/tests.py", line 162, in _test_func self.assertEqual(expected, result) AssertionError: Lists differ: [[(1, ([1], [2])), (2, ([1], [... != [] First list contains 3 additional elements. First extra element 0: [(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))] + [] - [[(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))], - [(1, ([1, 1, 1], [])), (2, ([1], [])), (4, ([], [1]))], - [('', ([1, 1], [1, 2])), ('a', ([1, 1], [1, 1])), ('b', ([1], [1]))]] ``` It also happened on Jenkins: http://spark-tests.appspot.com/builds/spark-branch-2.1-test-sbt-hadoop-2.7/120 It's because when the machine is overloaded, the timeout is not enough. This PR just increases the timeout to 30 seconds. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17597 from zsxwing/SPARK-20285.
* [SPARK-20280][CORE] FileStatusCache Weigher integer overflowBogdan Raducanu2017-04-102-13/+50
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Weigher.weigh needs to return Int but it is possible for an Array[FileStatus] to have size > Int.maxValue. To avoid this, the size is scaled down by a factor of 32. The maximumWeight of the cache is also scaled down by the same factor. ## How was this patch tested? New test in FileIndexSuite Author: Bogdan Raducanu <bogdan@databricks.com> Closes #17591 from bogdanrdc/SPARK-20280.
* [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish ↵Sean Owen2017-04-10126-299/+482
| | | | | | | | | | | | | | | | | | locale bug" causes Spark problems ## What changes were proposed in this pull request? Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17527 from srowen/SPARK-20156.
* [SPARK-20273][SQL] Disallow Non-deterministic Filter push-down into Join ↵Xiao Li2017-04-102-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | Conditions ## What changes were proposed in this pull request? ``` sql("SELECT t1.b, rand(0) as r FROM cachedData, cachedData t1 GROUP BY t1.b having r > 0.5").show() ``` We will get the following error: ``` Job aborted due to stage failure: Task 1 in stage 4.0 failed 1 times, most recent failure: Lost task 1.0 in stage 4.0 (TID 8, localhost, executor driver): java.lang.NullPointerException at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source) at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87) at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$1.apply(BroadcastNestedLoopJoinExec.scala:87) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:463) ``` Filters could be pushed down to the join conditions by the optimizer rule `PushPredicateThroughJoin`. However, Analyzer [blocks users to add non-deterministics conditions](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L386-L395) (For details, see the PR https://github.com/apache/spark/pull/7535). We should not push down non-deterministic conditions; otherwise, we need to explicitly initialize the non-deterministic expressions. This PR is to simply block it. ### How was this patch tested? Added a test case Author: Xiao Li <gatorsmile@gmail.com> Closes #17585 from gatorsmile/joinRandCondition.
* [SPARK-19518][SQL] IGNORE NULLS in first / last in SQLhyukjinkwon2017-04-103-1/+29
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to add `IGNORE NULLS` keyword in `first`/`last` in Spark's parser likewise http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions057.htm. This simply maps the keywords to existing `ignoreNullsExpr`. **Before** ```scala scala> sql("select first('a' IGNORE NULLS)").show() ``` ``` org.apache.spark.sql.catalyst.parser.ParseException: extraneous input 'NULLS' expecting {')', ','}(line 1, pos 24) == SQL == select first('a' IGNORE NULLS) ------------------------^^^ at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:210) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:112) at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:46) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:66) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:622) ... 48 elided ``` **After** ```scala scala> sql("select first('a' IGNORE NULLS)").show() ``` ``` +--------------+ |first(a, true)| +--------------+ | a| +--------------+ ``` ## How was this patch tested? Unit tests in `ExpressionParserSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17566 from HyukjinKwon/SPARK-19518.