aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-20159][SPARKR][SQL] Support all catalog API in RFelix Cheung2017-04-026-237/+569
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a set of catalog API in R ``` "currentDatabase", "listColumns", "listDatabases", "listFunctions", "listTables", "recoverPartitions", "refreshByPath", "refreshTable", "setCurrentDatabase", ``` https://github.com/apache/spark/pull/17483/files#diff-6929e6c5e59017ff954e110df20ed7ff ## How was this patch tested? manual tests, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17483 from felixcheung/rcatalog.
* [SPARK-20173][SQL][HIVE-THRIFTSERVER] Throw NullPointerException when ↵zuotingbing2017-04-021-1/+1
| | | | | | | | | | | | | | | | HiveThriftServer2 is shutdown ## What changes were proposed in this pull request? If the shutdown hook called before the variable `uiTab` is set , it will throw a NullPointerException. ## How was this patch tested? manual tests Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #17496 from zuotingbing/SPARK-HiveThriftServer2.
* [SPARK-20123][BUILD] SPARK_HOME variable might have spaces in it(e.g. $SPARK…zuotingbing2017-04-026-52/+52
| | | | | | | | | | | | | | | | JIRA Issue: https://issues.apache.org/jira/browse/SPARK-20123 ## What changes were proposed in this pull request? If $SPARK_HOME or $FWDIR variable contains spaces, then use "./dev/make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn" build spark will failed. ## How was this patch tested? manual tests Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #17452 from zuotingbing/spark-bulid.
* [SPARK-20143][SQL] DataType.fromJson should throw an exception with better ↵hyukjinkwon2017-04-022-1/+39
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | message ## What changes were proposed in this pull request? Currently, `DataType.fromJson` throws `scala.MatchError` or `java.util.NoSuchElementException` in some cases when the JSON input is invalid as below: ```scala DataType.fromJson(""""abcd"""") ``` ``` java.util.NoSuchElementException: key not found: abcd at ... ``` ```scala DataType.fromJson("""{"abcd":"a"}""") ``` ``` scala.MatchError: JObject(List((abcd,JString(a)))) (of class org.json4s.JsonAST$JObject) at ... ``` ```scala DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") ``` ``` scala.MatchError: JObject(List((a,JInt(123)))) (of class org.json4s.JsonAST$JObject) at ... ``` After this PR, ```scala DataType.fromJson(""""abcd"""") ``` ``` java.lang.IllegalArgumentException: Failed to convert the JSON string 'abcd' to a data type. at ... ``` ```scala DataType.fromJson("""{"abcd":"a"}""") ``` ``` java.lang.IllegalArgumentException: Failed to convert the JSON string '{"abcd":"a"}' to a data type. at ... ``` ```scala DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") at ... ``` ``` java.lang.IllegalArgumentException: Failed to convert the JSON string '{"a":123}' to a field. ``` ## How was this patch tested? Unit test added in `DataTypeSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17468 from HyukjinKwon/fromjson_exception.
* [SPARK-20186][SQL] BroadcastHint should use child's statswangzhenhua2017-04-012-2/+21
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `BroadcastHint` should use child's statistics and set `isBroadcastable` to true. ## How was this patch tested? Added a new stats estimation test for `BroadcastHint`. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17504 from wzhfy/broadcastHintEstimation.
* [SPARK-19148][SQL][FOLLOW-UP] do not expose the external table concept in ↵Xiao Li2017-04-011-10/+15
| | | | | | | | | | | | | | Catalog ### What changes were proposed in this pull request? After we renames `Catalog`.`createExternalTable` to `createTable` in the PR: https://github.com/apache/spark/pull/16528, we also need to deprecate the corresponding functions in `SQLContext`. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #17502 from gatorsmile/deprecateCreateExternalTable.
* [SPARK-20177] Document about compression way has some little detail ch…郭小龙 102076332017-04-011-2/+5
| | | | | | | | | | | | | | | | | | | | | | | | | …anges. ## What changes were proposed in this pull request? Document compression way little detail changes. 1.spark.eventLog.compress add 'Compression will use spark.io.compression.codec.' 2.spark.broadcast.compress add 'Compression will use spark.io.compression.codec.' 3,spark.rdd.compress add 'Compression will use spark.io.compression.codec.' 4.spark.io.compression.codec add 'event log describe'. eg Through the documents, I don't know what is compression mode about 'event log'. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn> Closes #17498 from guoxiaolongzte/SPARK-20177.
* [SPARK-20165][SS] Resolve state encoder's deserializer in driver in ↵Tathagata Das2017-03-316-21/+45
| | | | | | | | | | | | | | | | | | | FlatMapGroupsWithStateExec ## What changes were proposed in this pull request? - Encoder's deserializer must be resolved at the driver where the class is defined. Otherwise there are corner cases using nested classes where resolving at the executor can fail. - Fixed flaky test related to processing time timeout. The flakiness is caused because the test thread (that adds data to memory source) has a race condition with the streaming query thread. When testing the manual clock, the goal is to add data and increment clock together atomically, such that a trigger sees new data AND updated clock simultaneously (both or none). This fix adds additional synchronization in when adding data; it makes sure that the streaming query thread is waiting on the manual clock to be incremented (so no batch is currently running) before adding data. - Added`testQuietly` on some tests that generate a lot of error logs. ## How was this patch tested? Multiple runs on existing unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17488 from tdas/SPARK-20165.
* [SPARK-20160][SQL] Move ParquetConversions and OrcConversions Out Of ↵Xiao Li2017-04-017-122/+70
| | | | | | | | | | | | | | | | HiveSessionCatalog ### What changes were proposed in this pull request? `ParquetConversions` and `OrcConversions` should be treated as regular `Analyzer` rules. It is not reasonable to be part of `HiveSessionCatalog`. This PR also combines two rules `ParquetConversions` and `OrcConversions` to build a new rule `RelationConversions `. After moving these two rules out of HiveSessionCatalog, the next step is to clean up, rename and move `HiveMetastoreCatalog` because it is not related to the hive package any more. ### How was this patch tested? The existing test cases Author: Xiao Li <gatorsmile@gmail.com> Closes #17484 from gatorsmile/cleanup.
* [SPARK-20084][CORE] Remove internal.metrics.updatedBlockStatuses from ↵Ryan Blue2017-03-311-4/+11
| | | | | | | | | | | | | | | | history files. ## What changes were proposed in this pull request? Remove accumulator updates for internal.metrics.updatedBlockStatuses from SparkListenerTaskEnd entries in the history file. These can cause history files to grow to hundreds of GB because the value of the accumulator contains all tracked blocks. ## How was this patch tested? Current History UI tests cover use of the history file. Author: Ryan Blue <blue@apache.org> Closes #17412 from rdblue/SPARK-20084-remove-block-accumulator-info.
* [SPARK-20164][SQL] AnalysisException not tolerant of null query plan.Kunal Khamar2017-03-312-1/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The query plan in an `AnalysisException` may be `null` when an `AnalysisException` object is serialized and then deserialized, since `plan` is marked `transient`. Or when someone throws an `AnalysisException` with a null query plan (which should not happen). `def getMessage` is not tolerant of this and throws a `NullPointerException`, leading to loss of information about the original exception. The fix is to add a `null` check in `getMessage`. ## How was this patch tested? - Unit test Author: Kunal Khamar <kkhamar@outlook.com> Closes #17486 from kunalkhamar/spark-20164.
* [SPARK-20151][SQL] Account for partition pruning in scan metadataTime metricsReynold Xin2017-03-303-4/+18
| | | | | | | | | | | | ## What changes were proposed in this pull request? After SPARK-20136, we report metadata timing metrics in scan operator. However, that timing metric doesn't include one of the most important part of metadata, which is partition pruning. This patch adds that time measurement to the scan metrics. ## How was this patch tested? N/A - I tried adding a test in SQLMetricsSuite but it was extremely convoluted to the point that I'm not sure if this is worth it. Author: Reynold Xin <rxin@databricks.com> Closes #17476 from rxin/SPARK-20151.
* [SPARK-20121][SQL] simplify NullPropagation with NullIntolerantWenchen Fan2017-03-306-73/+39
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Instead of iterating all expressions that can return null for null inputs, we can just check `NullIntolerant`. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #17450 from cloud-fan/null.
* [SPARK-20127][CORE] few warning have been fixed which Intellij IDEA reported ↵Denis Bolshakov2017-03-3013-29/+22
| | | | | | | | | | | | | | Intellij IDEA ## What changes were proposed in this pull request? Few changes related to Intellij IDEA inspection. ## How was this patch tested? Changes were tested by existing unit tests Author: Denis Bolshakov <denis.bolshakov@onefactor.com> Closes #17458 from dbolshak/SPARK-20127.
* [DOCS][MINOR] Fixed a few typos in the Structured Streaming documentationSeigneurin, Alexis (CONT)2017-03-301-9/+9
| | | | | | | | | | | | | | | | | | Fixed a few typos. There is one more I'm not sure of: ``` Append mode uses watermark to drop old aggregation state. But the output of a windowed aggregation is delayed the late threshold specified in `withWatermark()` as by the modes semantics, rows can be added to the Result Table only once after they are ``` Not sure how to change `is delayed the late threshold`. Author: Seigneurin, Alexis (CONT) <Alexis.Seigneurin@capitalone.com> Closes #17443 from aseigneurin/typos.
* [SPARK-20096][SPARK SUBMIT][MINOR] Expose the right queue name not null if ↵Kent Yao2017-03-302-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | set by --conf or configure file ## What changes were proposed in this pull request? while submit apps with -v or --verbose, we can print the right queue name, but if we set a queue name with `spark.yarn.queue` by --conf or in the spark-default.conf, we just got `null` for the queue in Parsed arguments. ``` bin/spark-shell -v --conf spark.yarn.queue=thequeue Using properties file: /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf .... Adding default property: spark.yarn.queue=default Parsed arguments: master yarn deployMode client ... queue null .... verbose true Spark properties used, including those specified through --conf and those from the properties file /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf: spark.yarn.queue -> thequeue .... ``` ## How was this patch tested? ut and local verify Author: Kent Yao <yaooqinn@hotmail.com> Closes #17430 from yaooqinn/SPARK-20096.
* [SPARK-19999] Workaround JDK-8165231 to identify PPC64 architectures as ↵samelamin2017-03-301-12/+16
| | | | | | | | | | | | | | | | | | supporting unaligned access java.nio.Bits.unaligned() does not return true for the ppc64le arch. see https://bugs.openjdk.java.net/browse/JDK-8165231 ## What changes were proposed in this pull request? check architecture ## How was this patch tested? unit test Author: samelamin <hussam.elamin@gmail.com> Author: samelamin <sam_elamin@discovery.com> Closes #17472 from samelamin/SPARK-19999.
* [DOCS] Docs-only improvementsJacek Laskowski2017-03-3030-71/+68
| | | | | | | | | | | | | | | | …adoc ## What changes were proposed in this pull request? Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0). ## How was this patch tested? Local build Author: Jacek Laskowski <jacek@japila.pl> Closes #17417 from jaceklaskowski/window-expression-scaladoc.
* [SPARK-15354][CORE] Topology aware block replication strategiesShubham Chopra2017-03-304-32/+222
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Implementations of strategies for resilient block replication for different resource managers that replicate the 3-replica strategy used by HDFS, where the first replica is on an executor, the second replica within the same rack as the executor and a third replica on a different rack. The implementation involves providing two pluggable classes, one running in the driver that provides topology information for every host at cluster start and the second prioritizing a list of peer BlockManagerIds. The prioritization itself can be thought of an optimization problem to find a minimal set of peers that satisfy certain objectives and replicating to these peers first. The objectives can be used to express richer constraints over and above HDFS like 3-replica strategy. ## How was this patch tested? This patch was tested with unit tests for storage, along with new unit tests to verify prioritization behaviour. Author: Shubham Chopra <schopra31@bloomberg.net> Closes #13932 from shubhamchopra/PrioritizerStrategy.
* [SPARK-20107][DOC] Add ↵Yuming Wang2017-03-301-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version option to configuration.md ## What changes were proposed in this pull request? Add `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version` option to `configuration.md`. Set `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2` can speed up [HadoopMapReduceCommitProtocol.commitJob](https://github.com/apache/spark/blob/v2.1.0/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala#L121) for many output files. All cloudera's hadoop 2.6.0-cdh5.4.0 or higher versions(see: https://github.com/cloudera/hadoop-common/commit/1c1236182304d4075276c00c4592358f428bc433 and https://github.com/cloudera/hadoop-common/commit/16b2de27321db7ce2395c08baccfdec5562017f0) and apache's hadoop 2.7.0 or higher versions support this improvement. More see: 1. [MAPREDUCE-4815](https://issues.apache.org/jira/browse/MAPREDUCE-4815): Speed up FileOutputCommitter#commitJob for many output files. 2. [MAPREDUCE-6406](https://issues.apache.org/jira/browse/MAPREDUCE-6406): Update the default version for the property mapreduce.fileoutputcommitter.algorithm.version to 2. ## How was this patch tested? Manual test and exist tests. Author: Yuming Wang <wgyumg@gmail.com> Closes #17442 from wangyum/SPARK-20107.
* [MINOR][SPARKR] Add run command comment in exampleswm624@hotmail.com2017-03-292-0/+6
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There are two examples in r folder missing the run commands. In this PR, I just add the missing comment, which is consistent with other examples. ## How was this patch tested? Manual test. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #17474 from wangmiao1981/stat.
* [SPARK-20148][SQL] Extend the file commit API to allow subscribing to task ↵Eric Liang2017-03-293-7/+53
| | | | | | | | | | | | | | | | | | | | commit messages ## What changes were proposed in this pull request? The internal FileCommitProtocol interface returns all task commit messages in bulk to the implementation when a job finishes. However, it is sometimes useful to access those messages before the job completes, so that the driver gets incremental progress updates before the job finishes. This adds an `onTaskCommit` listener to the internal api. ## How was this patch tested? Unit tests. cc rxin Author: Eric Liang <ekl@databricks.com> Closes #17475 from ericl/file-commit-api-ext.
* [SPARK-20136][SQL] Add num files and metadata operation timing to scan ↵Reynold Xin2017-03-291-2/+16
| | | | | | | | | | | | | | | | | | operator metrics ## What changes were proposed in this pull request? This patch adds explicit metadata operation timing and number of files in data source metrics. Those would be useful to include for performance profiling. Screenshot of a UI with this change (num files and metadata time are new metrics): <img width="321" alt="screen shot 2017-03-29 at 12 29 28 am" src="https://cloud.githubusercontent.com/assets/323388/24443272/d4ea58c0-1416-11e7-8940-ecb69375554a.png"> ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #17465 from rxin/SPARK-20136.
* [SPARK-20146][SQL] fix comment missing issue for thrift serverbomeng2017-03-292-2/+14
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The column comment was missing while constructing the Hive TableSchema. This fix will preserve the original comment. ## How was this patch tested? I have added a new test case to test the column with/without comment. Author: bomeng <bmeng@us.ibm.com> Closes #17470 from bomeng/SPARK-20146.
* [SPARK-19088][SQL] Fix 2.10 build.Takuya UESHIN2017-03-291-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Commit 6c70a38 broke the build for scala 2.10. The commit uses some reflections which are not available in Scala 2.10. This PR fixes them. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@databricks.com> Closes #17473 from ueshin/issues/SPARK-19088.
* [SPARK-20120][SQL] spark-sql support silent modeYuming Wang2017-03-291-0/+5
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It is similar to Hive silent mode, just show the query result. see: [Hive LanguageManual+Cli](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli) and [the implementation of Hive silent mode](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L948-L950). This PR set the Logger level to `WARN` to get similar result. ## How was this patch tested? manual tests ![manual test spark sql silent mode](https://cloud.githubusercontent.com/assets/5399861/24390165/989b7780-13b9-11e7-8496-6e68f55757e3.gif) Author: Yuming Wang <wgyumg@gmail.com> Closes #17449 from wangyum/SPARK-20120.
* [SPARK-17075][SQL][FOLLOWUP] Add Estimation of Constant LiteralXiao Li2017-03-292-2/+124
| | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? `FalseLiteral` and `TrueLiteral` should have been eliminated by optimizer rule `BooleanSimplification`, but null literals might be added by optimizer rule `NullPropagation`. For safety, our filter estimation should handle all the eligible literal cases. Our optimizer rule BooleanSimplification is unable to remove the null literal in many cases. For example, `a < 0 or null`. Thus, we need to handle null literal in filter estimation. `Not` can be pushed down below `And` and `Or`. Then, we could see two consecutive `Not`, which need to be collapsed into one. Because of the limited expression support for filter estimation, we just need to handle the case `Not(null)` for avoiding incorrect error due to the boolean operation on null. For details, see below matrix. ``` not NULL = NULL NULL or false = NULL NULL or true = true NULL or NULL = NULL NULL and false = false NULL and true = NULL NULL and NULL = NULL ``` ### How was this patch tested? Added the test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17446 from gatorsmile/constantFilterEstimation.
* [SPARK-20009][SQL] Support DDL strings for defining schema in ↵Takeshi Yamamuro2017-03-295-25/+90
| | | | | | | | | | | | | | functions.from_json ## What changes were proposed in this pull request? This pr added `StructType.fromDDL` to convert a DDL format string into `StructType` for defining schemas in `functions.from_json`. ## How was this patch tested? Added tests in `JsonFunctionsSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17406 from maropu/SPARK-20009.
* [SPARK-20048][SQL] Cloning SessionState does not clone query execution listenersKunal Khamar2017-03-299-56/+111
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Bugfix from [SPARK-19540.](https://github.com/apache/spark/pull/16826) Cloning SessionState does not clone query execution listeners, so cloned session is unable to listen to events on queries. ## How was this patch tested? - Unit test Author: Kunal Khamar <kkhamar@outlook.com> Closes #17379 from kunalkhamar/clone-bugfix.
* [SPARK-19955][PYSPARK] Jenkins Python Conda based test.Holden Karau2017-03-293-28/+47
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Allow Jenkins Python tests to use the installed conda to test Python 2.7 support & test pip installability. ## How was this patch tested? Updated shell scripts, ran tests locally with installed conda, ran tests in Jenkins. Author: Holden Karau <holden@us.ibm.com> Closes #17355 from holdenk/SPARK-19955-support-python-tests-with-conda.
* [SPARK-20059][YARN] Use the correct classloader for HBaseCredentialProviderjerryshao2017-03-293-4/+15
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we use system classloader to find HBase jars, if it is specified by `--jars`, then it will be failed with ClassNotFound issue. So here changing to use child classloader. Also putting added jars and main jar into classpath of submitted application in yarn cluster mode, otherwise HBase jars specified with `--jars` will never be honored in cluster mode, and fetching tokens in client side will always be failed. ## How was this patch tested? Unit test and local verification. Author: jerryshao <sshao@hortonworks.com> Closes #17388 from jerryshao/SPARK-20059.
* [SPARK-19556][CORE] Do not encrypt block manager data in memory.Marcelo Vanzin2017-03-2920-270/+710
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change modifies the way block data is encrypted to make the more common cases faster, while penalizing an edge case. As a side effect of the change, all data that goes through the block manager is now encrypted only when needed, including the previous path (broadcast variables) where that did not happen. The way the change works is by not encrypting data that is stored in memory; so if a serialized block is in memory, it will only be encrypted once it is evicted to disk. The penalty comes when transferring that encrypted data from disk. If the data ends up in memory again, it is as efficient as before; but if the evicted block needs to be transferred directly to a remote executor, then there's now a performance penalty, since the code now uses a custom FileRegion implementation to decrypt the data before transferring. This also means that block data transferred between executors now is not encrypted (and thus relies on the network library encryption support for secrecy). Shuffle blocks are still transferred in encrypted form, since they're handled in a slightly different way by the code. This also keeps compatibility with existing external shuffle services, which transfer encrypted shuffle blocks, and avoids having to make the external service aware of encryption at all. The serialization and deserialization APIs in the SerializerManager now do not do encryption automatically; callers need to explicitly wrap their streams with an appropriate crypto stream before using those. As a result of these changes, some of the workarounds added in SPARK-19520 are removed here. Testing: a new trait ("EncryptionFunSuite") was added that provides an easy way to run a test twice, with encryption on and off; broadcast, block manager and caching tests were modified to use this new trait so that the existing tests exercise both encrypted and non-encrypted paths. I also ran some applications with encryption turned on to verify that they still work, including streaming tests that failed without the fix for SPARK-19520. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #17295 from vanzin/SPARK-19556.
* [SPARK-20134][SQL] SQLMetrics.postDriverMetricUpdates to simplify driver ↵Reynold Xin2017-03-295-17/+34
| | | | | | | | | | | | | | side metric updates ## What changes were proposed in this pull request? It is not super intuitive how to update SQLMetric on the driver side. This patch introduces a new SQLMetrics.postDriverMetricUpdates function to do that, and adds documentation to make it more obvious. ## How was this patch tested? Updated a test case to use this method. Author: Reynold Xin <rxin@databricks.com> Closes #17464 from rxin/SPARK-20134.
* [SPARK-20040][ML][PYTHON] pyspark wrapper for ChiSquareTestBago Amirbekian2017-03-285-12/+127
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A pyspark wrapper for spark.ml.stat.ChiSquareTest ## How was this patch tested? unit tests doctests Author: Bago Amirbekian <bago@databricks.com> Closes #17421 from MrBago/chiSquareTestWrapper.
* [SPARK-20043][ML] DecisionTreeModel: ImpurityCalculator builder fails for ↵颜发才(Yan Facai)2017-03-282-1/+15
| | | | | | | | | | | | | | uppercase impurity type Gini Fix bug: DecisionTreeModel can't recongnize Impurity "Gini" when loading TODO: + [x] add unit test + [x] fix the bug Author: 颜发才(Yan Facai) <facai.yan@gmail.com> Closes #17407 from facaiy/BUG/decision_tree_loader_failer_with_Gini_impurity.
* [SPARK-19868] conflict TasksetManager lead to spark stoppedliujianhui2017-03-282-8/+34
| | | | | | | | | | ## What changes were proposed in this pull request? We must set the taskset to zombie before the DAGScheduler handles the taskEnded event. It's possible the taskEnded event will cause the DAGScheduler to launch a new stage attempt (this happens when map output data was lost), and if this happens before the taskSet has been set to zombie, it will appear that we have conflicting task sets. Author: liujianhui <liujianhui@didichuxing> Closes #17208 from liujianhuiouc/spark-19868.
* [SPARK-20125][SQL] Dataset of type option of map does not workWenchen Fan2017-03-282-0/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? When we build the deserializer expression for map type, we will use `StaticInvoke` to call `ArrayBasedMapData.toScalaMap`, and declare the return type as `scala.collection.immutable.Map`. If the map is inside an Option, we will wrap this `StaticInvoke` with `WrapOption`, which requires the input to be `scala.collect.Map`. Ideally this should be fine, as `scala.collection.immutable.Map` extends `scala.collect.Map`, but our `ObjectType` is too strict about this, this PR fixes it. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17454 from cloud-fan/map.
* [SPARK-19995][YARN] Register tokens to current UGI to avoid re-issuing of ↵jerryshao2017-03-281-0/+3
| | | | | | | | | | | | | | | | | | | | tokens in yarn client mode ## What changes were proposed in this pull request? In the current Spark on YARN code, we will obtain tokens from provided services, but we're not going to add these tokens to the current user's credentials. This will make all the following operations to these services still require TGT rather than delegation tokens. This is unnecessary since we already got the tokens, also this will lead to failure in user impersonation scenario, because the TGT is granted by real user, not proxy user. So here changing to put all the tokens to the current UGI, so that following operations to these services will honor tokens rather than TGT, and this will further handle the proxy user issue mentioned above. ## How was this patch tested? Local verified in secure cluster. vanzin tgravescs mridulm dongjoon-hyun please help to review, thanks a lot. Author: jerryshao <sshao@hortonworks.com> Closes #17335 from jerryshao/SPARK-19995.
* [SPARK-20126][SQL] Remove HiveSessionStateHerman van Hovell2017-03-2814-193/+104
| | | | | | | | | | | | ## What changes were proposed in this pull request? Commit https://github.com/apache/spark/commit/ea361165e1ddce4d8aa0242ae3e878d7b39f1de2 moved most of the logic from the SessionState classes into an accompanying builder. This makes the existence of the `HiveSessionState` redundant. This PR removes the `HiveSessionState`. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17457 from hvanhovell/SPARK-20126.
* [SPARK-20124][SQL] Join reorder should keep the same order of final project ↵wangzhenhua2017-03-283-10/+31
| | | | | | | | | | | | | | | | | attributes ## What changes were proposed in this pull request? Join reorder algorithm should keep exactly the same order of output attributes in the top project. For example, if user want to select a, b, c, after reordering, we should output a, b, c in the same order as specified by user, instead of b, a, c or other orders. ## How was this patch tested? A new test case is added in `JoinReorderSuite`. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17453 from wzhfy/keepOrderInProject.
* [SPARK-20094][SQL] Preventing push down of IN subquery to Join operatorwangzhenhua2017-03-282-0/+26
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? TPCDS q45 fails becuase: `ReorderJoin` collects all predicates and try to put them into join condition when creating ordered join. If a predicate with an IN subquery (`ListQuery`) is in a join condition instead of a filter condition, `RewritePredicateSubquery.rewriteExistentialExpr` would fail to convert the subquery to an `ExistenceJoin`, and thus result in error. We should prevent push down of IN subquery to Join operator. ## How was this patch tested? Add a new test case in `FilterPushdownSuite`. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17428 from wzhfy/noSubqueryInJoinCond.
* [SPARK-20119][TEST-MAVEN] Fix the test case fail in ↵Xiao Li2017-03-281-1/+1
| | | | | | | | | | | | | | DataSourceScanExecRedactionSuite ### What changes were proposed in this pull request? Changed the pattern to match the first n characters in the location field so that the string truncation does not affect it. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #17448 from gatorsmile/fixTestCAse.
* [SPARK-19088][SQL] Optimize sequence type deserialization codegenMichal Senkyr2017-03-283-69/+54
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Optimization of arbitrary Scala sequence deserialization introduced by #16240. The previous implementation constructed an array which was then converted by `to`. This required two passes in most cases. This implementation attempts to remedy that by using `Builder`s provided by the `newBuilder` method on every Scala collection's companion object to build the resulting collection directly. Example codegen for simple `List` (obtained using `Seq(List(1)).toDS().map(identity).queryExecution.debug.codegen`): Before: ``` /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private boolean deserializetoobject_resultIsNull; /* 010 */ private java.lang.Object[] deserializetoobject_argValue; /* 011 */ private boolean MapObjects_loopIsNull1; /* 012 */ private int MapObjects_loopValue0; /* 013 */ private boolean deserializetoobject_resultIsNull1; /* 014 */ private scala.collection.generic.CanBuildFrom deserializetoobject_argValue1; /* 015 */ private UnsafeRow deserializetoobject_result; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder; /* 017 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter; /* 018 */ private scala.collection.immutable.List mapelements_argValue; /* 019 */ private UnsafeRow mapelements_result; /* 020 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder; /* 021 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter; /* 022 */ private scala.collection.immutable.List serializefromobject_argValue; /* 023 */ private UnsafeRow serializefromobject_result; /* 024 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 025 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 026 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter; /* 027 */ /* 028 */ public GeneratedIterator(Object[] references) { /* 029 */ this.references = references; /* 030 */ } /* 031 */ /* 032 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 033 */ partitionIndex = index; /* 034 */ this.inputs = inputs; /* 035 */ inputadapter_input = inputs[0]; /* 036 */ /* 037 */ deserializetoobject_result = new UnsafeRow(1); /* 038 */ this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32); /* 039 */ this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1); /* 040 */ /* 041 */ mapelements_result = new UnsafeRow(1); /* 042 */ this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32); /* 043 */ this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1); /* 044 */ /* 045 */ serializefromobject_result = new UnsafeRow(1); /* 046 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32); /* 047 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 048 */ this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(); /* 049 */ /* 050 */ } /* 051 */ /* 052 */ protected void processNext() throws java.io.IOException { /* 053 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 054 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 055 */ ArrayData inputadapter_value = inputadapter_row.getArray(0); /* 056 */ /* 057 */ deserializetoobject_resultIsNull = false; /* 058 */ /* 059 */ if (!deserializetoobject_resultIsNull) { /* 060 */ ArrayData deserializetoobject_value3 = null; /* 061 */ /* 062 */ if (!false) { /* 063 */ Integer[] deserializetoobject_convertedArray = null; /* 064 */ int deserializetoobject_dataLength = inputadapter_value.numElements(); /* 065 */ deserializetoobject_convertedArray = new Integer[deserializetoobject_dataLength]; /* 066 */ /* 067 */ int deserializetoobject_loopIndex = 0; /* 068 */ while (deserializetoobject_loopIndex < deserializetoobject_dataLength) { /* 069 */ MapObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex)); /* 070 */ MapObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex); /* 071 */ /* 072 */ if (MapObjects_loopIsNull1) { /* 073 */ throw new RuntimeException(((java.lang.String) references[0])); /* 074 */ } /* 075 */ if (false) { /* 076 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null; /* 077 */ } else { /* 078 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue0; /* 079 */ } /* 080 */ /* 081 */ deserializetoobject_loopIndex += 1; /* 082 */ } /* 083 */ /* 084 */ deserializetoobject_value3 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /* 085 */ } /* 086 */ boolean deserializetoobject_isNull2 = true; /* 087 */ java.lang.Object[] deserializetoobject_value2 = null; /* 088 */ if (!false) { /* 089 */ deserializetoobject_isNull2 = false; /* 090 */ if (!deserializetoobject_isNull2) { /* 091 */ Object deserializetoobject_funcResult = null; /* 092 */ deserializetoobject_funcResult = deserializetoobject_value3.array(); /* 093 */ if (deserializetoobject_funcResult == null) { /* 094 */ deserializetoobject_isNull2 = true; /* 095 */ } else { /* 096 */ deserializetoobject_value2 = (java.lang.Object[]) deserializetoobject_funcResult; /* 097 */ } /* 098 */ /* 099 */ } /* 100 */ deserializetoobject_isNull2 = deserializetoobject_value2 == null; /* 101 */ } /* 102 */ deserializetoobject_resultIsNull = deserializetoobject_isNull2; /* 103 */ deserializetoobject_argValue = deserializetoobject_value2; /* 104 */ } /* 105 */ /* 106 */ boolean deserializetoobject_isNull1 = deserializetoobject_resultIsNull; /* 107 */ final scala.collection.Seq deserializetoobject_value1 = deserializetoobject_resultIsNull ? null : scala.collection.mutable.WrappedArray.make(deserializetoobject_argValue); /* 108 */ deserializetoobject_isNull1 = deserializetoobject_value1 == null; /* 109 */ boolean deserializetoobject_isNull = true; /* 110 */ scala.collection.immutable.List deserializetoobject_value = null; /* 111 */ if (!deserializetoobject_isNull1) { /* 112 */ deserializetoobject_resultIsNull1 = false; /* 113 */ /* 114 */ if (!deserializetoobject_resultIsNull1) { /* 115 */ boolean deserializetoobject_isNull6 = false; /* 116 */ final scala.collection.generic.CanBuildFrom deserializetoobject_value6 = false ? null : scala.collection.immutable.List.canBuildFrom(); /* 117 */ deserializetoobject_isNull6 = deserializetoobject_value6 == null; /* 118 */ deserializetoobject_resultIsNull1 = deserializetoobject_isNull6; /* 119 */ deserializetoobject_argValue1 = deserializetoobject_value6; /* 120 */ } /* 121 */ /* 122 */ deserializetoobject_isNull = deserializetoobject_resultIsNull1; /* 123 */ if (!deserializetoobject_isNull) { /* 124 */ Object deserializetoobject_funcResult1 = null; /* 125 */ deserializetoobject_funcResult1 = deserializetoobject_value1.to(deserializetoobject_argValue1); /* 126 */ if (deserializetoobject_funcResult1 == null) { /* 127 */ deserializetoobject_isNull = true; /* 128 */ } else { /* 129 */ deserializetoobject_value = (scala.collection.immutable.List) deserializetoobject_funcResult1; /* 130 */ } /* 131 */ /* 132 */ } /* 133 */ deserializetoobject_isNull = deserializetoobject_value == null; /* 134 */ } /* 135 */ /* 136 */ boolean mapelements_isNull = true; /* 137 */ scala.collection.immutable.List mapelements_value = null; /* 138 */ if (!false) { /* 139 */ mapelements_argValue = deserializetoobject_value; /* 140 */ /* 141 */ mapelements_isNull = false; /* 142 */ if (!mapelements_isNull) { /* 143 */ Object mapelements_funcResult = null; /* 144 */ mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue); /* 145 */ if (mapelements_funcResult == null) { /* 146 */ mapelements_isNull = true; /* 147 */ } else { /* 148 */ mapelements_value = (scala.collection.immutable.List) mapelements_funcResult; /* 149 */ } /* 150 */ /* 151 */ } /* 152 */ mapelements_isNull = mapelements_value == null; /* 153 */ } /* 154 */ /* 155 */ if (mapelements_isNull) { /* 156 */ throw new RuntimeException(((java.lang.String) references[2])); /* 157 */ } /* 158 */ serializefromobject_argValue = mapelements_value; /* 159 */ /* 160 */ final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue); /* 161 */ serializefromobject_holder.reset(); /* 162 */ /* 163 */ // Remember the current cursor so that we can calculate how many bytes are /* 164 */ // written later. /* 165 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 166 */ /* 167 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 168 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 169 */ // grow the global buffer before writing data. /* 170 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 171 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 172 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 173 */ /* 174 */ } else { /* 175 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 176 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4); /* 177 */ /* 178 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 179 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 180 */ serializefromobject_arrayWriter.setNullInt(serializefromobject_index); /* 181 */ } else { /* 182 */ final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index); /* 183 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 184 */ } /* 185 */ } /* 186 */ } /* 187 */ /* 188 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 189 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 190 */ append(serializefromobject_result); /* 191 */ if (shouldStop()) return; /* 192 */ } /* 193 */ } /* 194 */ } ``` After: ``` /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIterator(references); /* 003 */ } /* 004 */ /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private boolean CollectObjects_loopIsNull1; /* 010 */ private int CollectObjects_loopValue0; /* 011 */ private UnsafeRow deserializetoobject_result; /* 012 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder; /* 013 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter; /* 014 */ private scala.collection.immutable.List mapelements_argValue; /* 015 */ private UnsafeRow mapelements_result; /* 016 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder; /* 017 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter; /* 018 */ private scala.collection.immutable.List serializefromobject_argValue; /* 019 */ private UnsafeRow serializefromobject_result; /* 020 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 021 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 022 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter; /* 023 */ /* 024 */ public GeneratedIterator(Object[] references) { /* 025 */ this.references = references; /* 026 */ } /* 027 */ /* 028 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 029 */ partitionIndex = index; /* 030 */ this.inputs = inputs; /* 031 */ inputadapter_input = inputs[0]; /* 032 */ /* 033 */ deserializetoobject_result = new UnsafeRow(1); /* 034 */ this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32); /* 035 */ this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1); /* 036 */ /* 037 */ mapelements_result = new UnsafeRow(1); /* 038 */ this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32); /* 039 */ this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1); /* 040 */ /* 041 */ serializefromobject_result = new UnsafeRow(1); /* 042 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32); /* 043 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 044 */ this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter(); /* 045 */ /* 046 */ } /* 047 */ /* 048 */ protected void processNext() throws java.io.IOException { /* 049 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 050 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 051 */ ArrayData inputadapter_value = inputadapter_row.getArray(0); /* 052 */ /* 053 */ scala.collection.immutable.List deserializetoobject_value = null; /* 054 */ /* 055 */ if (!false) { /* 056 */ int deserializetoobject_dataLength = inputadapter_value.numElements(); /* 057 */ scala.collection.mutable.Builder CollectObjects_builderValue2 = scala.collection.immutable.List$.MODULE$.newBuilder(); /* 058 */ CollectObjects_builderValue2.sizeHint(deserializetoobject_dataLength); /* 059 */ /* 060 */ int deserializetoobject_loopIndex = 0; /* 061 */ while (deserializetoobject_loopIndex < deserializetoobject_dataLength) { /* 062 */ CollectObjects_loopValue0 = (int) (inputadapter_value.getInt(deserializetoobject_loopIndex)); /* 063 */ CollectObjects_loopIsNull1 = inputadapter_value.isNullAt(deserializetoobject_loopIndex); /* 064 */ /* 065 */ if (CollectObjects_loopIsNull1) { /* 066 */ throw new RuntimeException(((java.lang.String) references[0])); /* 067 */ } /* 068 */ if (false) { /* 069 */ CollectObjects_builderValue2.$plus$eq(null); /* 070 */ } else { /* 071 */ CollectObjects_builderValue2.$plus$eq(CollectObjects_loopValue0); /* 072 */ } /* 073 */ /* 074 */ deserializetoobject_loopIndex += 1; /* 075 */ } /* 076 */ /* 077 */ deserializetoobject_value = (scala.collection.immutable.List) CollectObjects_builderValue2.result(); /* 078 */ } /* 079 */ /* 080 */ boolean mapelements_isNull = true; /* 081 */ scala.collection.immutable.List mapelements_value = null; /* 082 */ if (!false) { /* 083 */ mapelements_argValue = deserializetoobject_value; /* 084 */ /* 085 */ mapelements_isNull = false; /* 086 */ if (!mapelements_isNull) { /* 087 */ Object mapelements_funcResult = null; /* 088 */ mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue); /* 089 */ if (mapelements_funcResult == null) { /* 090 */ mapelements_isNull = true; /* 091 */ } else { /* 092 */ mapelements_value = (scala.collection.immutable.List) mapelements_funcResult; /* 093 */ } /* 094 */ /* 095 */ } /* 096 */ mapelements_isNull = mapelements_value == null; /* 097 */ } /* 098 */ /* 099 */ if (mapelements_isNull) { /* 100 */ throw new RuntimeException(((java.lang.String) references[2])); /* 101 */ } /* 102 */ serializefromobject_argValue = mapelements_value; /* 103 */ /* 104 */ final ArrayData serializefromobject_value = false ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue); /* 105 */ serializefromobject_holder.reset(); /* 106 */ /* 107 */ // Remember the current cursor so that we can calculate how many bytes are /* 108 */ // written later. /* 109 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 110 */ /* 111 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 112 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 113 */ // grow the global buffer before writing data. /* 114 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 115 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 116 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 117 */ /* 118 */ } else { /* 119 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 120 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4); /* 121 */ /* 122 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 123 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 124 */ serializefromobject_arrayWriter.setNullInt(serializefromobject_index); /* 125 */ } else { /* 126 */ final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index); /* 127 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 128 */ } /* 129 */ } /* 130 */ } /* 131 */ /* 132 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 133 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 134 */ append(serializefromobject_result); /* 135 */ if (shouldStop()) return; /* 136 */ } /* 137 */ } /* 138 */ } ``` Benchmark results before: ``` OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH AMD A10-4600M APU with Radeon(tm) HD Graphics collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Seq 269 / 370 0.0 269125.8 1.0X List 154 / 176 0.0 154453.5 1.7X mutable.Queue 210 / 233 0.0 209691.6 1.3X ``` Benchmark results after: ``` OpenJDK 64-Bit Server VM 1.8.0_112-b15 on Linux 4.8.13-1-ARCH AMD A10-4600M APU with Radeon(tm) HD Graphics collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Seq 255 / 316 0.0 254697.3 1.0X List 152 / 177 0.0 152410.0 1.7X mutable.Queue 213 / 235 0.0 213470.0 1.2X ``` ## How was this patch tested? ```bash ./build/mvn -DskipTests clean package && ./dev/run-tests ``` Additionally in Spark Shell: ```scala case class QueueClass(q: scala.collection.immutable.Queue[Int]) spark.createDataset(Seq(List(1,2,3))).map(x => QueueClass(scala.collection.immutable.Queue(x: _*))).map(_.q.dequeue).collect ``` Author: Michal Senkyr <mike.senkyr@gmail.com> Closes #16541 from michalsenkyr/dataset-seq-builder.
* [SPARK-20100][SQL] Refactor SessionState initializationHerman van Hovell2017-03-2813-572/+547
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current SessionState initialization code path is quite complex. A part of the creation is done in the SessionState companion objects, a part of the creation is one inside the SessionState class, and a part is done by passing functions. This PR refactors this code path, and consolidates SessionState initialization into a builder class. This SessionState will not do any initialization and just becomes a place holder for the various Spark SQL internals. This also lays the ground work for two future improvements: 1. This provides us with a start for removing the `HiveSessionState`. Removing the `HiveSessionState` would also require us to move resource loading into a separate class, and to (re)move metadata hive. 2. This makes it easier to customize the Spark Session. Currently you will need to create a custom version of the builder. I have added hooks to facilitate this. A future step will be to create a semi stable API on top of this. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17433 from hvanhovell/SPARK-20100.
* [SPARK-19876][SS] Follow up: Refactored BatchCommitLog to simplify logicTathagata Das2017-03-273-12/+19
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Existing logic seemingly writes null to the BatchCommitLog, even though it does additional checks to write '{}' (valid json) to the log. This PR simplifies the logic by disallowing use of `log.add(batchId, metadata)` and instead using `log.add(batchId)`. No question of specifying metadata, so no confusion related to null. ## How was this patch tested? Existing tests pass. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17444 from tdas/SPARK-19876-1.
* [SPARK-19803][CORE][TEST] Proactive replication test failuresShubham Chopra2017-03-283-12/+29
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Executors cache a list of their peers that is refreshed by default every minute. The cached stale references were randomly being used for replication. Since those executors were removed from the master, they did not occur in the block locations as reported by the master. This was fixed by 1. Refreshing peer cache in the block manager before trying to pro-actively replicate. This way the probability of replicating to a failed executor is eliminated. 2. Explicitly stopping the block manager in the tests. This shuts down the RPC endpoint use by the block manager. This way, even if a block manager tries to replicate using a stale reference, the replication logic should take care of refreshing the list of peers after failure. ## How was this patch tested? Tested manually Author: Shubham Chopra <schopra31@bloomberg.net> Author: Kay Ousterhout <kayousterhout@gmail.com> Author: Shubham Chopra <shubhamchopra@users.noreply.github.com> Closes #17325 from shubhamchopra/SPARK-19803.
* [MINOR][SPARKR] Move 'Data type mapping between R and Spark' to right place ↵Yanbo Liang2017-03-271-69/+69
| | | | | | | | | | | | | | | | | in SparkR doc. Section ```Data type mapping between R and Spark``` was put in the wrong place in SparkR doc currently, we should move it to a separate section. ## What changes were proposed in this pull request? Before this PR: ![image](https://cloud.githubusercontent.com/assets/1962026/24340911/bc01a532-126a-11e7-9a08-0d60d13a547c.png) After this PR: ![image](https://cloud.githubusercontent.com/assets/1962026/24340938/d9d32a9a-126a-11e7-8891-d2f5b46e0c71.png) Author: Yanbo Liang <ybliang8@gmail.com> Closes #17440 from yanboliang/sparkr-doc.
* [SPARK-20105][TESTS][R] Add tests for checkType and type string in ↵hyukjinkwon2017-03-271-0/+53
| | | | | | | | | | | | | | | | | | structField in R ## What changes were proposed in this pull request? It seems `checkType` and the type string in `structField` are not being tested closely. This string format currently seems SparkR-specific (see https://github.com/apache/spark/blob/d1f6c64c4b763c05d6d79ae5497f298dc3835f3e/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala#L93-L131) but resembles SQL type definition. Therefore, it seems nicer if we test positive/negative cases in R side. ## How was this patch tested? Unit tests in `test_sparkSQL.R`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17439 from HyukjinKwon/r-typestring-tests.
* [SPARK-20102] Fix nightly packaging and RC packaging scripts w/ two minor ↵Josh Rosen2017-03-272-5/+4
| | | | | | | | | | | | | | | | | | | build fixes ## What changes were proposed in this pull request? The master snapshot publisher builds are currently broken due to two minor build issues: 1. For unknown reasons, the LFTP `mkdir -p` command began throwing errors when the remote directory already exists. This change of behavior might have been caused by configuration changes in the ASF's SFTP server, but I'm not entirely sure of that. To work around this problem, this patch updates the script to ignore errors from the `lftp mkdir -p` commands. 2. The PySpark `setup.py` file references a non-existent `pyspark.ml.stat` module, causing Python packaging to fail by complaining about a missing directory. The fix is to simply drop that line from the setup script. ## How was this patch tested? The LFTP fix was tested by manually running the failing commands on AMPLab Jenkins against the ASF SFTP server. The PySpark fix was tested locally. Author: Josh Rosen <joshrosen@databricks.com> Closes #17437 from JoshRosen/spark-20102.
* [SPARK-20088] Do not create new SparkContext in SparkR createSparkContextHossein2017-03-271-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Instead of creating new `JavaSparkContext` we use `SparkContext.getOrCreate`. ## How was this patch tested? Existing tests Author: Hossein <hossein@databricks.com> Closes #17423 from falaki/SPARK-20088.