aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-14237][SQL] De-duplicate partition value appending logic in various ↵Cheng Lian2016-05-049-53/+74
| | | | | | | | | | | | | | | | | | | | | | | | | | | | buildReader() implementations ## What changes were proposed in this pull request? Currently, various `FileFormat` data sources share approximately the same code for partition value appending. This PR tries to eliminate this duplication. A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a default implementation that appends partition values to `InternalRow`s produced by the reader function returned by `buildReader()`. Special data sources like Parquet, which implements partition value appending inside `buildReader()` because of the vectorized reader, and the Text data source, which doesn't support partitioning, override `buildReaderWithPartitionValues()` and simply delegate to `buildReader()`. This PR brings two benefits: 1. Apparently, it de-duplicates partition value appending logic 2. Now the reader function returned by `buildReader()` is only required to produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't override `buildReaderWithPartitionValues()`. Because the safe-to-unsafe conversion is also performed while appending partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to implement since they no longer need to access private APIs involving `UnsafeRow`. ## How was this patch tested? Existing tests should do the work. Author: Cheng Lian <lian@databricks.com> Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending.
* [SPARK-15107][SQL] Allow varying # iterations by test case in BenchmarkReynold Xin2016-05-033-67/+93
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch changes our micro-benchmark util to allow setting different iteration numbers for different test cases. For some of our benchmarks, turning off whole-stage codegen can make the runtime 20X slower, making it very difficult to run a large number of times without substantially shortening the input cardinality. With this change, I set the default num iterations to 2 for whole stage codegen off, and 5 for whole stage codegen on. I also updated some results. ## How was this patch tested? N/A - this is a test util. Author: Reynold Xin <rxin@databricks.com> Closes #12884 from rxin/SPARK-15107.
* [SPARK-15095][SQL] remove HiveSessionHook from ThriftServerDavies Liu2016-05-032-57/+0
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove HiveSessionHook ## How was this patch tested? No tests needed. Author: Davies Liu <davies@databricks.com> Closes #12881 from davies/remove_hooks.
* [SPARK-14414][SQL] Make DDL exceptions more consistentAndrew Or2016-05-0320-435/+141
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Just a bunch of small tweaks on DDL exception messages. ## How was this patch tested? `DDLCommandSuite` et al. Author: Andrew Or <andrew@databricks.com> Closes #12853 from andrewor14/make-exceptions-consistent.
* [SPARK-15097][SQL] make Dataset.sqlContext a stable identifier for importsKoert Kuipers2016-05-032-1/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Make Dataset.sqlContext a lazy val so that its a stable identifier and can be used for imports. Now this works again: import someDataset.sqlContext.implicits._ ## How was this patch tested? Add unit test to DatasetSuite that uses the import show above. Author: Koert Kuipers <koert@tresata.com> Closes #12877 from koertkuipers/feat-sqlcontext-stable-import.
* [SPARK-15084][PYTHON][SQL] Use builder pattern to create SparkSession in ↵Dongjoon Hyun2016-05-032-21/+105
| | | | | | | | | | | | | | | | PySpark. ## What changes were proposed in this pull request? This is a python port of corresponding Scala builder pattern code. `sql.py` is modified as a target example case. ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12860 from dongjoon-hyun/SPARK-15084.
* [SPARK-14645][MESOS] Fix python running on cluster mode mesos to have non ↵Timothy Chen2016-05-031-2/+3
| | | | | | | | | | | | | | | | local uris ## What changes were proposed in this pull request? Fix SparkSubmit to allow non-local python uris ## How was this patch tested? Manually tested with mesos-spark-dispatcher Author: Timothy Chen <tnachen@gmail.com> Closes #12403 from tnachen/enable_remote_python.
* [SPARK-14422][SQL] Improve handling of optional configs in SQLConfSandeep Singh2016-05-034-10/+25
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Create a new API for handling Optional Configs in SQLConf. Right now `getConf` for `OptionalConfigEntry[T]` returns value of type `T`, if doesn't exist throws an exception. Add new method `getOptionalConf`(suggestions on naming) which will now returns value of type `Option[T]`(so if doesn't exist it returns `None`). ## How was this patch tested? Add test and ran tests locally. Author: Sandeep Singh <sandeep@techaddict.me> Closes #12846 from techaddict/SPARK-14422.
* [MINOR][DOC] Fixed some python snippets in mllib data types documentation.Shuai Lin2016-05-031-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Some python snippets is using scala imports and comments. ## How was this patch tested? Generated the docs locally with `SKIP_API=1 jekyll build` and viewed the changes in the browser. Author: Shuai Lin <linshuai2012@gmail.com> Closes #12869 from lins05/fix-mllib-python-snippets.
* [SPARK-15104] Fix spacing in log lineAndrew Ash2016-05-031-1/+1
| | | | | | | | | | | | Otherwise get logs that look like this (note no space before NODE_LOCAL) ``` INFO [2016-05-03 21:18:51,477] org.apache.spark.scheduler.TaskSetManager: Starting task 0.0 in stage 101.0 (TID 7029, localhost, partition 0,NODE_LOCAL, 1894 bytes) ``` Author: Andrew Ash <andrew@andrewash.com> Closes #12880 from ash211/patch-7.
* [SQL-15102][SQL] remove delegation token support from ThriftServerDavies Liu2016-05-031-58/+7
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? These API is only useful for Hadoop, may not work for Spark SQL. The APIs is kept for source compatibility. ## How was this patch tested? No unit tests needed. Author: Davies Liu <davies@databricks.com> Closes #12878 from davies/remove_delegate.
* [SPARK-15056][SQL] Parse Unsupported Sampling Syntax and Issue Better Exceptionsgatorsmile2016-05-033-3/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | | #### What changes were proposed in this pull request? Compared with the current Spark parser, there are two extra syntax are supported in Hive for sampling - In `On` clauses, `rand()` is used for indicating sampling on the entire row instead of an individual column. For example, ```SQL SELECT * FROM source TABLESAMPLE(BUCKET 3 OUT OF 32 ON rand()) s; ``` - Users can specify the total length to be read. For example, ```SQL SELECT * FROM source TABLESAMPLE(100M) s; ``` Below is the link for references: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Sampling This PR is to parse and capture these two extra syntax, and issue a better error message. #### How was this patch tested? Added test cases to verify the thrown exceptions Author: gatorsmile <gatorsmile@gmail.com> Closes #12838 from gatorsmile/bucketOnRand.
* [SPARK-14973][ML] The CrossValidator and TrainValidationSplit miss the seed ↵yinxusen2016-05-035-18/+30
| | | | | | | | | | | | | | | | | | when saving and loading ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14973 Add seed support when saving/loading of CrossValidator and TrainValidationSplit. ## How was this patch tested? Spark unit test. Author: yinxusen <yinxusen@gmail.com> Closes #12825 from yinxusen/SPARK-14973.
* [SPARK-15095][SQL] drop binary mode in ThriftServerDavies Liu2016-05-033-47/+14
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR drop the support for binary mode in ThriftServer, only HTTP mode is supported now, to reduce the maintain burden. The code to support binary mode is still kept, just in case if we want it in future. ## How was this patch tested? Updated tests to use HTTP mode. Author: Davies Liu <davies@databricks.com> Closes #12876 from davies/hide_binary.
* [SPARK-15073][SQL] Hide SparkSession constructor from the publicAndrew Or2016-05-034-12/+19
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Users should use the builder pattern instead. ## How was this patch tested? Jenks. Author: Andrew Or <andrew@databricks.com> Closes #12873 from andrewor14/spark-session-constructor.
* [SPARK-11316] coalesce doesn't handle UnionRDD with partial locality properlyThomas Graves2016-05-032-62/+165
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? coalesce doesn't handle UnionRDD with partial locality properly. I had a user who had a UnionRDD that was made up of mapPartitionRDD without preferred locations and a checkpointedRDD with preferred locations (getting from hdfs). It took the driver over 20 minutes to setup the groups and put the partitions into those groups before it even started any tasks. Even perhaps worse is it didn't end up with the number of partitions he was asking for because it didn't put a partition in each of the groups properly. The changes in this patch get rid of a n^2 while loop that was causing the 20 minutes, it properly distributes the partitions to have at least one per group, and it changes from using the rotation iterator which got the preferred locations many times to get all the preferred locations once up front. Note that the n^2 while loop that I removed in setupGroups took so long because all of the partitions with preferred locations were already assigned to group, so it basically looped through every single one and wasn't ever able to assign it. At the time I had 960 partitions with preferred locations and 1020 without and did the outer while loop 319 times because that is the # of groups left to create. Note that each of those times through the inner while loop is going off to hdfs to get the block locations, so this is extremely inefficient. ## How was the this patch tested? Added unit tests for this case and ran existing ones that applied to make sure no regressions. Also manually tested on the users production job to make sure it fixed their issue. It created the proper number of partitions and now it takes about 6 seconds rather then 20 minutes. I did also run some basic manual tests with spark-shell doing coalesced to smaller number, same number, and then greater with shuffle. Author: Thomas Graves <tgraves@prevailsail.corp.gq1.yahoo.com> Closes #11327 from tgravescs/SPARK-11316.
* [SPARK-14521] [SQL] StackOverflowError in Kryo when executing TPC-DSyzhou20012016-05-032-42/+129
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Observed stackOverflowError in Kryo when executing TPC-DS Query27. Spark thrift server disables kryo reference tracking (if not specified in conf). When "spark.kryo.referenceTracking" is set to true explicitly in spark-defaults.conf, query executes successfully. The root cause is that the TaskMemoryManager inside MemoryConsumer and LongToUnsafeRowMap were not transient and thus were serialized and broadcast around from within LongHashedRelation, which could potentially cause circular reference inside Kryo. But the TaskMemoryManager is per task and should not be passed around at the first place. This fix makes it transient. ## How was this patch tested? core/test, hive/test, sql/test, catalyst/test, dev/lint-scala, org.apache.spark.sql.hive.execution.HiveCompatibilitySuite, dev/scalastyle, manual test of TBC-DS Query 27 with 1GB data but without the "limit 100" which would cause a NPE due to SPARK-14752. Author: yzhou2001 <yzhou_1999@yahoo.com> Closes #12598 from yzhou2001/master.
* [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruptionDevaraj K2016-05-031-1/+25
| | | | | | | | | | | | ## What changes were proposed in this pull request? Resetting the task interruption status before updating the task status. ## How was this patch tested? I have verified it manually by running multiple applications, Executor doesn't crash and updates the status to the driver without any exceptions with the patch changes. Author: Devaraj K <devaraj@apache.org> Closes #12031 from devaraj-kavali/SPARK-14234.
* [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to ↵Zheng Tan2016-05-031-26/+5
| | | | | | | | | | | | avoid dead lock ## What changes were proposed in this pull request? In some cases, fine-grained lock have race condition with class-loader lock and have caused dead lock issue. It is safe to drop this fine grained lock and load all classes by single class-loader lock. Author: Zheng Tan <zheng.tan@hulu.com> Closes #12857 from tankkyo/master.
* [SPARK-15082][CORE] Improve unit test coverage for AccumulatorV2Sandeep Singh2016-05-031-1/+60
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Added tests for ListAccumulator and LegacyAccumulatorWrapper, test for ListAccumulator is one similar to old Collection Accumulators ## How was this patch tested? Ran tests locally. cc rxin Author: Sandeep Singh <sandeep@techaddict.me> Closes #12862 from techaddict/SPARK-15082.
* [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in ↵François Garillot2016-05-035-5/+11
| | | | | | | | | | | | incremental versions of reduceByWindow - that reduceFunc and invReduceFunc should be associative - that the intermediate result in iterated applications of inverseReduceFunc is its first argument Author: François Garillot <francois@garillot.net> Closes #8103 from huitseeker/issue/invReduceFuncDoc.
* [SPARK-15087][CORE][SQL] Remove AccumulatorV2.localValue and keep only valueSandeep Singh2016-05-0311-38/+30
| | | | | | | | | | | | ## What changes were proposed in this pull request? Remove AccumulatorV2.localValue and keep only value ## How was this patch tested? existing tests Author: Sandeep Singh <sandeep@techaddict.me> Closes #12865 from techaddict/SPARK-15087.
* [SPARK-14860][TESTS] Create a new Waiter in reset to bypass an issue of ↵Shixiong Zhu2016-05-031-9/+3
| | | | | | | | | | | | | | | | ScalaTest's Waiter.wait ## What changes were proposed in this pull request? This PR updates `QueryStatusCollector.reset` to create Waiter instead of calling `await(1 milliseconds)` to bypass an ScalaTest's issue that Waiter.await may block forever. ## How was this patch tested? I created a local stress test to call codes in `test("event ordering")` 100 times. It cannot pass without this patch. Author: Shixiong Zhu <shixiong@databricks.com> Closes #12623 from zsxwing/flaky-test.
* [SPARK-14716][SQL] Added support for partitioning in FileStreamSinkTathagata Das2016-05-039-54/+605
| | | | | | | | | | | | | | | | | # What changes were proposed in this pull request? Support partitioning in the file stream sink. This is implemented using a new, but simpler code path for writing parquet files - both unpartitioned and partitioned. This new code path does not use Output Committers, as we will eventually write the file names to the metadata log for "committing" them. This patch duplicates < 100 LOC from the WriterContainer. But its far simpler that WriterContainer as it does not involve output committing. In addition, it introduces the new APIs in FileFormat and OutputWriterFactory in an attempt to simplify the APIs (not have Job in the `FileFormat` API, not have bucket and other stuff in the `OutputWriterFactory.newInstance()` ). # Tests - New unit tests to test the FileStreamSinkWriter for partitioned and unpartitioned files - New unit test to partially test the FileStreamSink for partitioned files (does not test recovery of partition column data, as that requires change in the StreamFileCatalog, future PR). - Updated FileStressSuite to test number of records read from partitioned output files. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #12409 from tdas/streaming-partitioned-parquet.
* [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queriesLiwei Lin2016-05-032-4/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Since we've been processing continuous queries in separate threads, the call sites are then `run at <unknown>:0`. It's not wrong but provides very little information; in addition, we can not distinguish two queries only from their call sites. This patch fixes this. ### Before [Jobs Tab] ![s1a](https://cloud.githubusercontent.com/assets/15843379/14766101/a47246b2-0a30-11e6-8d81-06a9a600113b.png) [SQL Tab] ![s1b](https://cloud.githubusercontent.com/assets/15843379/14766102/a4750226-0a30-11e6-9ada-773d977d902b.png) ### After [Jobs Tab] ![s2a](https://cloud.githubusercontent.com/assets/15843379/14766104/a89705b6-0a30-11e6-9830-0d40ec68527b.png) [SQL Tab] ![s2b](https://cloud.githubusercontent.com/assets/15843379/14766103/a8966728-0a30-11e6-8e4d-c2e326400478.png) ## How was this patch tested? Manually checks - see screenshots above. Author: Liwei Lin <lwlin7@gmail.com> Closes #12650 from lw-lin/fix-call-site.
* [SPARK-15088] [SQL] Remove SparkSqlSerializerReynold Xin2016-05-032-118/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes SparkSqlSerializer. I believe this is now dead code. ## How was this patch tested? Removed a test case related to it. Author: Reynold Xin <rxin@databricks.com> Closes #12864 from rxin/SPARK-15088.
* [SPARK-15091][SPARKR] Fix warnings and a failure in SparkR test cases with ↵Sun Rui2016-05-034-10/+9
| | | | | | | | | | | | | | testthat version 1.0.1 ## What changes were proposed in this pull request? Fix warnings and a failure in SparkR test cases with testthat version 1.0.1 ## How was this patch tested? SparkR unit test cases. Author: Sun Rui <sunrui2016@gmail.com> Closes #12867 from sun-rui/SPARK-15091.
* [SPARK-14971][ML][PYSPARK] PySpark ML Params setter code clean upYanbo Liang2016-05-0310-219/+110
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? PySpark ML Params setter code clean up. For examples, ```setInputCol``` can be simplified from ``` self._set(inputCol=value) return self ``` to: ``` return self._set(inputCol=value) ``` This is a pretty big sweeps, and we cleaned wherever possible. ## How was this patch tested? Exist unit tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #12749 from yanboliang/spark-14971.
* [SPARK-15057][GRAPHX] Remove stale TODO comment for making `enum` in ↵Dongjoon Hyun2016-05-031-1/+0
| | | | | | | | | | | | | | | | GraphGenerators ## What changes were proposed in this pull request? This PR removes a stale TODO comment in `GraphGenerators.scala` ## How was this patch tested? Just comment removed. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12839 from dongjoon-hyun/SPARK-15057.
* [SPARK-14897][CORE] Upgrade Jetty to latest version of 8Sean Owen2016-05-031-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Update Jetty 8.1 to the latest 2016/02 release, from a 2013/10 release, for security and bug fixes. This does not resolve the JIRA necessarily, as it's still worth considering an update to 9.3. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #12842 from srowen/SPARK-14897.
* [SPARK-15081] Move AccumulatorV2 and subclasses into util packageReynold Xin2016-05-0330-34/+44
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch moves AccumulatorV2 and subclasses into util package. ## How was this patch tested? Updated relevant tests. Author: Reynold Xin <rxin@databricks.com> Closes #12863 from rxin/SPARK-15081.
* [SPARK-15053][BUILD] Fix Java Lint errors on Hive-Thriftserver moduleDongjoon Hyun2016-05-0330-71/+63
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This issue fixes or hides 181 Java linter errors introduced by SPARK-14987 which copied hive service code from Hive. We had better clean up these errors before releasing Spark 2.0. - Fix UnusedImports (15 lines), RedundantModifier (14 lines), SeparatorWrap (9 lines), MethodParamPad (6 lines), FileTabCharacter (5 lines), ArrayTypeStyle (3 lines), ModifierOrder (3 lines), RedundantImport (1 line), CommentsIndentation (1 line), UpperEll (1 line), FallThrough (1 line), OneStatementPerLine (1 line), NewlineAtEndOfFile (1 line) errors. - Ignore `LineLength` errors under `hive/service/*` (118 lines). - Ignore `MethodName` error in `PasswdAuthenticationProvider.java` (1 line). - Ignore `NoFinalizer` error in `ThreadWithGarbageCleanup.java` (1 line). ## How was this patch tested? After passing Jenkins building, run `dev/lint-java` manually. ```bash $ dev/lint-java Checkstyle checks passed. ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12831 from dongjoon-hyun/SPARK-15053.
* [MINOR][DOCS] Fix type Information in Quick Start and Programming GuideSandeep Singh2016-05-032-5/+5
| | | | | | Author: Sandeep Singh <sandeep@techaddict.me> Closes #12841 from techaddict/improve_docs_1.
* [SPARK-6717][ML] Clear shuffle files after checkpointing in ALSHolden Karau2016-05-033-45/+195
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When ALS is run with a checkpoint interval, during the checkpoint materialize the current state and cleanup the previous shuffles (non-blocking). ## How was this patch tested? Existing ALS unit tests, new ALS checkpoint cleanup unit tests added & shuffle files checked after ALS w/checkpointing run. Author: Holden Karau <holden@us.ibm.com> Author: Holden Karau <holden@pigscanfly.ca> Closes #11919 from holdenk/SPARK-6717-clear-shuffle-files-after-checkpointing-in-ALS.
* [SPARK-13749][SQL][FOLLOW-UP] Faster pivot implementation for many distinct ↵Andrew Ray2016-05-022-3/+20
| | | | | | | | | | | | | | | | values with two phase aggregation ## What changes were proposed in this pull request? This is a follow up PR for #11583. It makes 3 lazy vals into just vals and adds unit test coverage. ## How was this patch tested? Existing unit tests and additional unit tests. Author: Andrew Ray <ray.andrew@gmail.com> Closes #12861 from aray/fast-pivot-follow-up.
* [SPARK-15079] Support average/count/sum in Long/DoubleAccumulatorReynold Xin2016-05-025-101/+181
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes AverageAccumulator and adds the ability to compute average to LongAccumulator and DoubleAccumulator. The patch also improves documentation for the two accumulators. ## How was this patch tested? Added unit tests for this. Author: Reynold Xin <rxin@databricks.com> Closes #12858 from rxin/SPARK-15079.
* [SPARK-14685][CORE] Document heritability of localPropertiesMarcin Tustin2016-05-023-2/+40
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This updates the java-/scala- doc for setLocalProperty to document heritability of localProperties. This also adds tests for that behaviour. ## How was this patch tested? Tests pass. New tests were added. Author: Marcin Tustin <marcin.tustin@gmail.com> Closes #12455 from marcintustin/SPARK-14685.
* [SPARK-15077][SQL] Use a fair lock to avoid thread starvation in StreamExecutionShixiong Zhu2016-05-021-27/+52
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Right now `StreamExecution.awaitBatchLock` uses an unfair lock. `StreamExecution.awaitOffset` may run too long and fail some test because `StreamExecution.constructNextBatch` keeps getting the lock. See: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/865/testReport/junit/org.apache.spark.sql.streaming/FileStreamSourceStressTestSuite/file_source_stress_test/ This PR uses a fair ReentrantLock to resolve the thread starvation issue. ## How was this patch tested? Modified `FileStreamSourceStressTestSuite.test("file source stress test")` to run the test codes 100 times locally. It always fails because of timeout without this patch. Author: Shixiong Zhu <shixiong@databricks.com> Closes #12852 from zsxwing/SPARK-15077.
* [SPARK-15062][SQL] fix list type infer serializer issuebomeng2016-05-022-5/+21
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Make serializer correctly inferred if the input type is `List[_]`, since `List[_]` is type of `Seq[_]`, before it was matched to different case (`case t if definedByConstructorParams(t)`). ## How was this patch tested? New test case was added. Author: bomeng <bmeng@us.ibm.com> Closes #12849 from bomeng/SPARK-15062.
* [SPARK-15047][SQL] Cleanup SQL ParserHerman van Hovell2016-05-026-77/+31
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR addresses a few minor issues in SQL parser: - Removes some unused rules and keywords in the grammar. - Removes code path for fallback SQL parsing (was needed for Hive native parsing). - Use `UnresolvedGenerator` instead of hard-coding `Explode` & `JsonTuple`. - Adds a more generic way of creating error messages for unsupported Hive features. - Use `visitFunctionName` as much as possible. - Interpret a `CatalogColumn`'s `DataType` directly instead of parsing it again. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12826 from hvanhovell/SPARK-15047.
* [SPARK-15050][SQL] Put CSV and JSON options as Python csv and json function ↵hyukjinkwon2016-05-021-77/+155
| | | | | | | | | | | | | | | | | | | parameters ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-15050 This PR adds function parameters for Python API for reading and writing `csv()`. ## How was this patch tested? This was tested by `./dev/run_tests`. Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #12834 from HyukjinKwon/SPARK-15050.
* [SPARK-14747][SQL] Add assertStreaming/assertNoneStreaming checks in ↵Liwei Lin2016-05-022-5/+210
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DataFrameWriter ## Problem If an end user happens to write code mixed with continuous-query-oriented methods and non-continuous-query-oriented methods: ```scala ctx.read .format("text") .stream("...") // continuous query .write .text("...") // non-continuous query; should be startStream() here ``` He/she would get this somehow confusing exception: > Exception in thread "main" java.lang.AssertionError: assertion failed: No plan for FileSource[./continuous_query_test_input] at scala.Predef$.assert(Predef.scala:170) at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at ... ## What changes were proposed in this pull request? This PR adds checks for continuous-query-oriented methods and non-continuous-query-oriented methods in `DataFrameWriter`: <table> <tr> <td align="center"></td> <td align="center"><strong>can be called on continuous query?</strong></td> <td align="center"><strong>can be called on non-continuous query?</strong></td> </tr> <tr> <td align="center">mode</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">trigger</td> <td align="center">yes</td> <td align="center"></td> </tr> <tr> <td align="center">format</td> <td align="center">yes</td> <td align="center">yes</td> </tr> <tr> <td align="center">option/options</td> <td align="center">yes</td> <td align="center">yes</td> </tr> <tr> <td align="center">partitionBy</td> <td align="center">yes</td> <td align="center">yes</td> </tr> <tr> <td align="center">bucketBy</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">sortBy</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">save</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">queryName</td> <td align="center">yes</td> <td align="center"></td> </tr> <tr> <td align="center">startStream</td> <td align="center">yes</td> <td align="center"></td> </tr> <tr> <td align="center">insertInto</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">saveAsTable</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">jdbc</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">json</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">parquet</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">orc</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">text</td> <td align="center"></td> <td align="center">yes</td> </tr> <tr> <td align="center">csv</td> <td align="center"></td> <td align="center">yes</td> </tr> </table> After this PR's change, the friendly exception would be: > Exception in thread "main" org.apache.spark.sql.AnalysisException: text() can only be called on non-continuous queries; at org.apache.spark.sql.DataFrameWriter.assertNotStreaming(DataFrameWriter.scala:678) at org.apache.spark.sql.DataFrameWriter.text(DataFrameWriter.scala:629) at ss.SSDemo$.main(SSDemo.scala:47) ## How was this patch tested? dedicated unit tests were added Author: Liwei Lin <lwlin7@gmail.com> Closes #12521 from lw-lin/dataframe-writer-check.
* [SPARK-14785] [SQL] Support correlated scalar subqueriesHerman van Hovell2016-05-027-39/+195
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In this PR we add support for correlated scalar subqueries. An example of such a query is: ```SQL select * from tbl1 a where a.value > (select max(value) from tbl2 b where b.key = a.key) ``` The implementation adds the `RewriteCorrelatedScalarSubquery` rule to the Optimizer. This rule plans these subqueries using `LEFT OUTER` joins. It currently supports rewrites for `Project`, `Aggregate` & `Filter` logical plans. I could not find a well defined semantics for the use of scalar subqueries in an `Aggregate`. The current implementation currently evaluates the scalar subquery *before* aggregation. This means that you either have to make scalar subquery part of the grouping expression, or that you have to aggregate it further on. I am open to suggestions on this. The implementation currently forces the uniqueness of a scalar subquery by enforcing that it is aggregated and that the resulting column is wrapped in an `AggregateExpression`. ## How was this patch tested? Added tests to `SubquerySuite`. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12822 from hvanhovell/SPARK-14785.
* [SPARK-12928][SQL] Oracle FLOAT datatype is not properly handled when ↵poolis2016-05-022-0/+15
| | | | | | | | | | | reading via JDBC The contribution is my original work and that I license the work to the project under the project's open source license. Author: poolis <gmichalopoulos@gmail.com> Author: Greg Michalopoulos <gmichalopoulos@gmail.com> Closes #10899 from poolis/spark-12928.
* [SPARK-15052][SQL] Use builder pattern to create SparkSessionReynold Xin2016-05-021-4/+127
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch creates a builder pattern for creating SparkSession. The new code is unused and mostly deadcode. I'm putting it up here for feedback. There are a few TODOs that can be done as follow-up pull requests: - [ ] Update tests to use this - [ ] Update examples to use this - [ ] Clean up SQLContext code w.r.t. this one (i.e. SparkSession shouldn't call into SQLContext.getOrCreate; it should be the other way around) - [ ] Remove SparkSession.withHiveSupport - [ ] Disable the old constructor (by making it private) so the only way to start a SparkSession is through this builder pattern ## How was this patch tested? Part of the future pull request is to clean this up and switch existing tests to use this. Author: Reynold Xin <rxin@databricks.com> Closes #12830 from rxin/sparksession-builder.
* [SPARK-15054] Deprecate old accumulator APIReynold Xin2016-05-023-10/+20
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch deprecates the old accumulator API. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #12832 from rxin/SPARK-15054.
* [SPARK-13745] [SQL] Support columnar in memory representation on Big Endian ↵Pete Robbins2016-05-024-22/+110
| | | | | | | | | | | | | | | | platforms ## What changes were proposed in this pull request? parquet datasource and ColumnarBatch tests fail on big-endian platforms This patch adds support for the little-endian byte arrays being correctly interpreted on a big-endian platform ## How was this patch tested? Spark test builds ran on big endian z/Linux and regression build on little endian amd64 Author: Pete Robbins <robbinspg@gmail.com> Closes #12397 from robbinspg/master.
* [SPARK-14781] [SQL] support nested predicate subqueryDavies Liu2016-05-0214-61/+345
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In order to support nested predicate subquery, this PR introduce an internal join type ExistenceJoin, which will emit all the rows from left, plus an additional column, which presents there are any rows matched from right or not (it's not null-aware right now). This additional column could be used to replace the subquery in Filter. In theory, all the predicate subquery could use this join type, but it's slower than LeftSemi and LeftAnti, so it's only used for nested subquery (subquery inside OR). For example, the following SQL: ```sql SELECT a FROM t WHERE EXISTS (select 0) OR EXISTS (select 1) ``` This PR also fix a bug in predicate subquery push down through join (they should not). Nested null-aware subquery is still not supported. For example, `a > 3 OR b NOT IN (select bb from t)` After this, we could run TPCDS query Q10, Q35, Q45 ## How was this patch tested? Added unit tests. Author: Davies Liu <davies@databricks.com> Closes #12820 from davies/or_exists.
* [SPARK-14830][SQL] Add RemoveRepetitionFromGroupExpressions optimizer.Dongjoon Hyun2016-05-022-2/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR aims to optimize GroupExpressions by removing repeating expressions. `RemoveRepetitionFromGroupExpressions` is added. **Before** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, (1 + a#0)#7, (A#0 + 1)#8, (1 + A#0)#9, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6,(1 + a#0) AS (1 + a#0)#7,(A#0 + 1) AS (A#0 + 1)#8,(1 + A#0) AS (1 + A#0)#9], functions=[], output=[(a#0 + 1)#6,(1 + a#0)#7,(A#0 + 1)#8,(1 + A#0)#9]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` **After** ```scala scala> sql("select a+1 from values 1,2 T(a) group by a+1, 1+a, A+1, 1+A").explain() == Physical Plan == WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1)#6], functions=[], output=[(a + 1)#5]) : +- INPUT +- Exchange hashpartitioning((a#0 + 1)#6, 200), None +- WholeStageCodegen : +- TungstenAggregate(key=[(a#0 + 1) AS (a#0 + 1)#6], functions=[], output=[(a#0 + 1)#6]) : +- INPUT +- LocalTableScan [a#0], [[1],[2]] ``` ## How was this patch tested? Pass the Jenkins tests (with a new testcase) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #12590 from dongjoon-hyun/SPARK-14830.
* [SPARK-14579][SQL] Fix the race condition in ↵Shixiong Zhu2016-05-022-5/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | StreamExecution.processAllAvailable again ## What changes were proposed in this pull request? #12339 didn't fix the race condition. MemorySinkSuite is still flaky: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.2/814/testReport/junit/org.apache.spark.sql.streaming/MemorySinkSuite/registering_as_a_table/ Here is an execution order to reproduce it. | Time |Thread 1 | MicroBatchThread | |:-------------:|:-------------:|:-----:| | 1 | | `MemorySink.getOffset` | | 2 | | availableOffsets ++= newData (availableOffsets is not changed here) | | 3 | addData(newData) | | | 4 | Set `noNewData` to `false` in processAllAvailable | | | 5 | | `dataAvailable` returns `false` | | 6 | | noNewData = true | | 7 | `noNewData` is true so just return | | | 8 | assert results and fail | | | 9 | | `dataAvailable` returns true so process the new batch | This PR expands the scope of `awaitBatchLock.synchronized` to eliminate the above race. ## How was this patch tested? test("stress test"). It always failed before this patch. And it will pass after applying this patch. Ignore this test in the PR as it takes several minutes to finish. Author: Shixiong Zhu <shixiong@databricks.com> Closes #12582 from zsxwing/SPARK-14579-2.