aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17342][WEBUI] Style of event timeline is brokenKousuke Saruta2016-09-022-32/+31
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-15373 (#13158) updated the version of vis.js to 4.16.1. As of 4.0.0, some class was renamed like 'timeline to vis-timeline' but that ticket didn't care and now style is broken. In this PR, I've restored the style by modifying `timeline-view.css` and `timeline-view.js`. ## How was this patch tested? manual tests. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) * Before <img width="1258" alt="2016-09-01 1 38 31" src="https://cloud.githubusercontent.com/assets/4736016/18141311/fddf1bac-6ff3-11e6-935f-28b389073b39.png"> * After <img width="1256" alt="2016-09-01 3 30 19" src="https://cloud.githubusercontent.com/assets/4736016/18141394/49af65dc-6ff4-11e6-8640-70e20300f3c3.png"> Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #14900 from sarutak/SPARK-17342.
* [SPARK-16926][SQL] Add unit test to compare table and partition column metadata.Brian Cho2016-09-021-0/+35
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add unit test for changes made in PR #14515. It makes sure that a newly created table has the same number of columns in table and partition metadata. This test fails before the changes introduced in #14515. ## How was this patch tested? Run new unit test. Author: Brian Cho <bcho@fb.com> Closes #14930 from dafrista/partition-metadata-unit-test.
* [SPARK-16302][SQL] Set the right number of partitions for reading data from ↵Lianhui Wang2016-09-011-1/+4
| | | | | | | | | | | | | a local collection. follow #13137 This pr sets the right number of partitions when reading data from a local collection. Query 'val df = Seq((1, 2)).toDF("key", "value").count' always use defaultParallelism tasks. So it causes run many empty or small tasks. Manually tested and checked. Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #13979 from lianhuiwang/localTable-Parallel.
* [SPARK-16619] Add shuffle service metrics entry in monitoring docsYangyang Liu2016-09-011-0/+1
| | | | | | | | After change [SPARK-16405](https://github.com/apache/spark/pull/14080), we need to update docs by adding shuffle service metrics entry in currently supporting metrics list. Author: Yangyang Liu <yangyangliu@fb.com> Closes #14254 from lovexi/yangyang-monitoring-doc.
* [SPARK-16525] [SQL] Enable Row Based HashMap in HashAggregateExecQifan Pu2016-09-018-119/+326
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is the second step for the following feature: For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields). In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs. In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`. ## How was this patch tested? Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite` Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series. Author: Qifan Pu <qifan.pu@gmail.com> Author: ooq <qifan.pu@gmail.com> Closes #14176 from ooq/rowbasedfastaggmap-pr2.
* [SPARK-17355] Workaround for HIVE-14684 / HiveResultSetMetaData.isSigned ↵Josh Rosen2016-09-011-1/+10
| | | | | | | | | | | | | | | | | | | | | | | exception ## What changes were proposed in this pull request? Attempting to use Spark SQL's JDBC data source against the Hive ThriftServer results in a `java.sql.SQLException: Method` not supported exception from `org.apache.hive.jdbc.HiveResultSetMetaData.isSigned`. Here are two user reports of this issue: - https://stackoverflow.com/questions/34067686/spark-1-5-1-not-working-with-hive-jdbc-1-2-0 - https://stackoverflow.com/questions/32195946/method-not-supported-in-spark I have filed [HIVE-14684](https://issues.apache.org/jira/browse/HIVE-14684) to attempt to fix this in Hive by implementing the isSigned method, but in the meantime / for compatibility with older JDBC drivers I think we should add special-case error handling to work around this bug. This patch updates `JDBCRDD`'s `ResultSetMetadata` to schema conversion to catch the "Method not supported" exception from Hive and return `isSigned = true`. I believe that this is safe because, as far as I know, Hive does not support unsigned numeric types. ## How was this patch tested? Tested manually against a Spark Thrift Server. Author: Josh Rosen <joshrosen@databricks.com> Closes #14911 from JoshRosen/hive-jdbc-workaround.
* [SPARK-16461][SQL] Support partition batch pruning with `<=>` predicate in ↵hyukjinkwon2016-09-012-0/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | InMemoryTableScanExec ## What changes were proposed in this pull request? It seems `EqualNullSafe` filter was missed for batch pruneing partitions in cached tables. It seems supporting this improves the performance roughly 5 times faster. Running the codes below: ```scala test("Null-safe equal comparison") { val N = 20000000 val df = spark.range(N).repartition(20) val benchmark = new Benchmark("Null-safe equal comparison", N) df.createOrReplaceTempView("t") spark.catalog.cacheTable("t") sql("select id from t where id <=> 1").collect() benchmark.addCase("Null-safe equal comparison", 10) { _ => sql("select id from t where id <=> 1").collect() } benchmark.run() } ``` produces the results below: **Before:** ``` Running benchmark: Null-safe equal comparison Running case: Null-safe equal comparison Stopped after 10 iterations, 2098 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5 Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Null-safe equal comparison 204 / 210 98.1 10.2 1.0X ``` **After:** ``` Running benchmark: Null-safe equal comparison Running case: Null-safe equal comparison Stopped after 10 iterations, 478 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5 Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Null-safe equal comparison 42 / 48 474.1 2.1 1.0X ``` ## How was this patch tested? Unit tests in `PartitionBatchPruningSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14117 from HyukjinKwon/SPARK-16461.
* [SPARK-16732][SQL] Remove unused codes in ↵Yucai Yu2016-09-011-4/+0
| | | | | | | | | | | | | | | subexpressionEliminationForWholeStageCodegen ## What changes were proposed in this pull request? Some codes in subexpressionEliminationForWholeStageCodegen are never used actually. Remove them using this PR. ## How was this patch tested? Local unit tests. Author: Yucai Yu <yucai.yu@intel.com> Closes #14366 from yucai/subExpr_unused_codes.
* [SPARK-16926] [SQL] Remove partition columns from partition metadata.Brian Cho2016-09-011-1/+7
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This removes partition columns from column metadata of partitions to match tables. A change introduced in SPARK-14388 removed partition columns from the column metadata of tables, but not for partitions. This causes TableReader to believe that the schema is different between table and partition, and create an unnecessary conversion object inspector in TableReader. ## How was this patch tested? Existing unit tests. Author: Brian Cho <bcho@fb.com> Closes #14515 from dafrista/partition-columns-metadata.
* [SPARK-16533][HOTFIX] Fix compilation on Scala 2.10.Marcelo Vanzin2016-09-011-1/+3
| | | | | | | | | No idea why it was failing (the needed import was there), but this makes things work. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14925 from vanzin/SPARK-16533.
* [SPARK-17331][CORE][MLLIB] Avoid allocating 0-length arraysSean Owen2016-09-0111-20/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid allocating some 0-length arrays, esp. in UTF8String, and by using Array.empty in Scala over Array[T]() ## How was this patch tested? Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #14895 from srowen/SPARK-17331.
* [SPARK-17263][SQL] Add hexadecimal literal parsingHerman van Hovell2016-09-016-25/+93
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`. If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`. Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data. ## How was this patch tested? Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14832 from hvanhovell/SPARK-17263.
* [SPARK-16533][CORE] resolve deadlocking in driver when executors dieAngus Gerry2016-09-019-138/+169
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request reverts the changes made as a part of #14605, which simply side-steps the deadlock issue. Instead, I propose the following approach: * Use `scheduleWithFixedDelay` when calling `ExecutorAllocationManager.schedule` for scheduling executor requests. The intent of this is that if invocations are delayed beyond the default schedule interval on account of lock contention, then we avoid a situation where calls to `schedule` are made back-to-back, potentially releasing and then immediately reacquiring these locks - further exacerbating contention. * Replace a number of calls to `askWithRetry` with `ask` inside of message handling code in `CoarseGrainedSchedulerBackend` and its ilk. This allows us queue messages with the relevant endpoints, release whatever locks we might be holding, and then block whilst awaiting the response. This change is made at the cost of being able to retry should sending the message fail, as retrying outside of the lock could easily cause race conditions if other conflicting messages have been sent whilst awaiting a response. I believe this to be the lesser of two evils, as in many cases these RPC calls are to process local components, and so failures are more likely to be deterministic, and timeouts are more likely to be caused by lock contention. ## How was this patch tested? Existing tests, and manual tests under yarn-client mode. Author: Angus Gerry <angolon@gmail.com> Closes #14710 from angolon/SPARK-16533.
* [SPARK-17271][SQL] Remove redundant `semanticEquals()` from `SortOrder`Tejas Patil2016-09-011-3/+0
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Removing `semanticEquals()` from `SortOrder` because it can use the `semanticEquals()` provided by its parent class (`Expression`). This was as per suggestion by cloud-fan at https://github.com/apache/spark/pull/14841/files/7192418b3a26a14642fc04fc92bf496a954ffa5d#r77106801 ## How was this patch tested? Ran the test added in https://github.com/apache/spark/pull/14841 Author: Tejas Patil <tejasp@fb.com> Closes #14910 from tejasapatil/SPARK-17271_remove_semantic_ordering.
* [SPARK-17257][SQL] the physical plan of CREATE TABLE or CTAS should take ↵Wenchen Fan2016-09-015-104/+78
| | | | | | | | | | | | | | | | CatalogTable ## What changes were proposed in this pull request? This is kind of a follow-up of https://github.com/apache/spark/pull/14482 . As we put `CatalogTable` in the logical plan directly, it makes sense to let physical plans take `CatalogTable` directly, instead of extracting some fields of `CatalogTable` in planner and then construct a new `CatalogTable` in physical plan. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14823 from cloud-fan/create-table.
* [SPARK-17353][SPARK-16943][SPARK-16942][SQL] Fix multiple bugs in CREATE ↵gatorsmile2016-09-013-14/+273
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | TABLE LIKE command ### What changes were proposed in this pull request? The existing `CREATE TABLE LIKE` command has multiple issues: - The generated table is non-empty when the source table is a data source table. The major reason is the data source table is using the table property `path` to store the location of table contents. Currently, we keep it unchanged. Thus, we still create the same table with the same location. - The table type of the generated table is `EXTERNAL` when the source table is an external Hive Serde table. Currently, we explicitly set it to `MANAGED`, but Hive is checking the table property `EXTERNAL` to decide whether the table is `EXTERNAL` or not. (See https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1407-L1408) Thus, the created table is still `EXTERNAL`. - When the source table is a `VIEW`, the metadata of the generated table contains the original view text and view original text. So far, this does not break anything, but it could cause something wrong in Hive. (For example, https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1405-L1406) - The issue regarding the table `comment`. To follow what Hive does, the table comment should be cleaned, but the column comments should be still kept. - The `INDEX` table is not supported. Thus, we should throw an exception in this case. - `owner` should not be retained. `ToHiveTable` set it [here](https://github.com/apache/spark/blob/e679bc3c1cd418ef0025d2ecbc547c9660cac433/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L793) no matter which value we set in `CatalogTable`. We set it to an empty string for avoiding the confusing output in Explain. - Add a support for temp tables - Like Hive, we should not copy the table properties from the source table to the created table, especially for the statistics-related properties, which could be wrong in the created table. - `unsupportedFeatures` should not be copied from the source table. The created table does not have these unsupported features. - When the type of source table is a view, the target table is using the default format of data source tables: `spark.sql.sources.default`. This PR is to fix the above issues. ### How was this patch tested? Improve the test coverage by adding more test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14531 from gatorsmile/createTableLike.
* fixed typosSeigneurin, Alexis (CONT)2016-09-011-2/+2
| | | | | | | | fixed 2 typos Author: Seigneurin, Alexis (CONT) <Alexis.Seigneurin@capitalone.com> Closes #14877 from aseigneurin/fix-typo-2.
* [SPARK-16283][SQL] Implements percentile_approx aggregation function which ↵Sean Zhong2016-09-016-2/+893
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | supports partial aggregation. ## What changes were proposed in this pull request? This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`. ### Syntax: ``` # Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory. percentile_approx(col, percentage [, accuracy]) # Returns percentile value array at given percentage value array percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) ``` ### Features: 1. This function supports partial aggregation. 2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint. 3. This function supports window function aggregation. ### Example usages: ``` ## Returns the 25th percentile value, with default accuracy SELECT percentile_approx(col, 0.25) FROM table ## Returns an array of percentile value (25th, 50th, 75th), with default accuracy SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table ## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error SELECT percentile_approx(col, 0.25, 100) FROM table ## Returns the 25th, and 50th percentile values, with custom accuracy value 100 SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table ``` ### NOTE: 1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)` 2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal. ## How was this patch tested? Unit test, and Sql query test. ## Acknowledgement 1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue. Author: Sean Zhong <seanzhong@databricks.com> Closes #14868 from clockfly/appro_percentile_try_2.
* [SPARK-17329][BUILD] Don't build PRs with -Pyarn unless YARN code changedSean Owen2016-09-012-5/+6
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Only build PRs with -Pyarn if YARN code was modified. ## How was this patch tested? Jenkins tests (will look to verify whether -Pyarn was included in the PR builder for this one.) Author: Sean Owen <sowen@cloudera.com> Closes #14892 from srowen/SPARK-17329.
* [SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with class ↵Shixiong Zhu2016-08-311-0/+9
| | | | | | | | | | | | | | | | | | defined in repl again ## What changes were proposed in this pull request? After digging into the logs, I noticed the failure is because in this test, it starts a local cluster with 2 executors. However, when SparkContext is created, executors may be still not up. When one of the executor is not up during running the job, the blocks won't be replicated. This PR just adds a wait loop before running the job to fix the flaky test. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #14905 from zsxwing/SPARK-17318-2.
* revert PR#10896 and PR#14865Wenchen Fan2016-09-018-277/+223
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? according to the discussion in the original PR #10896 and the new approach PR #14876 , we decided to revert these 2 PRs and go with the new approach. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14909 from cloud-fan/revert.
* [SPARK-17241][SPARKR][MLLIB] SparkR spark.glm should have configurable ↵Xin Ren2016-08-314-5/+55
| | | | | | | | | | | | | | | | | | regularization parameter https://issues.apache.org/jira/browse/SPARK-17241 ## What changes were proposed in this pull request? Spark has configurable L2 regularization parameter for generalized linear regression. It is very important to have them in SparkR so that users can run ridge regression. ## How was this patch tested? Test manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14856 from keypointt/SPARK-17241.
* [SPARKR][MINOR] Fix windowPartitionBy exampleJunyang Qian2016-08-311-6/+6
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The usage in the original example is incorrect. This PR fixes it. ## How was this patch tested? Manual test. Author: Junyang Qian <junyangq@databricks.com> Closes #14903 from junyangq/SPARKR-FixWindowPartitionByDoc.
* [SPARK-16581][SPARKR] Fix JVM API tests in SparkRShivaram Venkataraman2016-08-311-11/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove cleanup.jobj test. Use JVM wrapper API for other test cases. ## How was this patch tested? Run R unit tests with testthat 1.0 Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #14904 from shivaram/sparkr-jvm-tests-fix.
* [SPARK-17316][TESTS] Fix MesosCoarseGrainedSchedulerBackendSuiteShixiong Zhu2016-08-311-0/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The master is broken because #14882 didn't run mesos tests. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14902 from zsxwing/hotfix.
* [SPARK-17326][SPARKR] Fix tests with HiveContext in SparkR not to be skipped ↵hyukjinkwon2016-08-311-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | always ## What changes were proposed in this pull request? Currently, `HiveContext` in SparkR is not being tested and always skipped. This is because the initiation of `TestHiveContext` is being failed due to trying to load non-existing data paths (test tables). This is introduced from https://github.com/apache/spark/pull/14005 This enables the tests with SparkR. ## How was this patch tested? Manually, **Before** (on Mac OS) ``` ... Skipped ------------------------------------------------------------------------ 1. create DataFrame from RDD (test_sparkSQL.R#200) - Hive is not build with SparkSQL, skipped 2. test HiveContext (test_sparkSQL.R#1041) - Hive is not build with SparkSQL, skipped 3. read/write ORC files (test_sparkSQL.R#1748) - Hive is not build with SparkSQL, skipped 4. enableHiveSupport on SparkSession (test_sparkSQL.R#2480) - Hive is not build with SparkSQL, skipped 5. sparkJars tag in SparkContext (test_Windows.R#21) - This test is only for Windows, skipped ... ``` **After** (on Mac OS) ``` ... Skipped ------------------------------------------------------------------------ 1. sparkJars tag in SparkContext (test_Windows.R#21) - This test is only for Windows, skipped ... ``` Please refer the tests below (on Windows) - Before: https://ci.appveyor.com/project/HyukjinKwon/spark/build/45-test123 - After: https://ci.appveyor.com/project/HyukjinKwon/spark/build/46-test123 Author: hyukjinkwon <gurwls223@gmail.com> Closes #14889 from HyukjinKwon/SPARK-17326.
* [SPARK-17332][CORE] Make Java Loggers static membersSean Owen2016-08-3127-27/+29
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Make all Java Loggers static members ## How was this patch tested? Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #14896 from srowen/SPARK-17332.
* [SPARK-17316][CORE] Make CoarseGrainedSchedulerBackend.removeExecutor ↵Shixiong Zhu2016-08-311-8/+9
| | | | | | | | | | | | | | | | | | non-blocking ## What changes were proposed in this pull request? StandaloneSchedulerBackend.executorRemoved is a blocking call right now. It may cause some deadlock since it's called inside StandaloneAppClient.ClientEndpoint. This PR just changed CoarseGrainedSchedulerBackend.removeExecutor to be non-blocking. It's safe since the only two usages (StandaloneSchedulerBackend and YarnSchedulerEndpoint) don't need the return value). ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14882 from zsxwing/SPARK-17316.
* [SPARK-17320] add build_profile_flags entry to mesos build moduleMichael Gummelt2016-08-311-0/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? add build_profile_flags entry to mesos build module ## How was this patch tested? unit tests Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14885 from mgummelt/mesos-profile.
* [MINOR][SPARKR] Verbose build comment in WINDOWS.md rather than promoting ↵hyukjinkwon2016-08-311-1/+11
| | | | | | | | | | | | | | | | | | default build without Hive ## What changes were proposed in this pull request? This PR fixes `WINDOWS.md` to imply referring other profiles in http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn rather than directly pointing to run `mvn -DskipTests -Psparkr package` without Hive supports. ## How was this patch tested? Manually, <img width="626" alt="2016-08-31 6 01 08" src="https://cloud.githubusercontent.com/assets/6477701/18122549/f6297b2c-6fa4-11e6-9b5e-fd4347355d87.png"> Author: hyukjinkwon <gurwls223@gmail.com> Closes #14890 from HyukjinKwon/minor-build-r.
* [SPARK-17180][SPARK-17309][SPARK-17323][SQL] create AlterViewAsCommand to ↵Wenchen Fan2016-08-313-60/+157
| | | | | | | | | | | | | | | | | | | | | | handle ALTER VIEW AS ## What changes were proposed in this pull request? Currently we use `CreateViewCommand` to implement ALTER VIEW AS, which has 3 bugs: 1. SPARK-17180: ALTER VIEW AS should alter temp view if view name has no database part and temp view exists 2. SPARK-17309: ALTER VIEW AS should issue exception if view does not exist. 3. SPARK-17323: ALTER VIEW AS should keep the previous table properties, comment, create_time, etc. The root cause is, ALTER VIEW AS is quite different from CREATE VIEW, we need different code path to handle them. However, in `CreateViewCommand`, there is no way to distinguish ALTER VIEW AS and CREATE VIEW, we have to introduce extra flag. But instead of doing this, I think a more natural way is to separate the ALTER VIEW AS logic into a new command. ## How was this patch tested? new tests in SQLViewSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #14874 from cloud-fan/minor4.
* [SPARK-17178][SPARKR][SPARKSUBMIT] Allow to set sparkr shell command through ↵Jeff Zhang2016-08-314-2/+32
| | | | | | | | | | | | | | | | | | | --conf ## What changes were proposed in this pull request? Allow user to set sparkr shell command through --conf spark.r.shell.command ## How was this patch tested? Unit test is added and also verify it manually through ``` bin/sparkr --master yarn-client --conf spark.r.shell.command=/usr/local/bin/R ``` Author: Jeff Zhang <zjffdu@apache.org> Closes #14744 from zjffdu/SPARK-17178.
* [SPARK-15985][SQL] Eliminate redundant cast from an array without null or a ↵Kazuaki Ishizaki2016-08-313-0/+76
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | map without null ## What changes were proposed in this pull request? This PR eliminates redundant cast from an `ArrayType` with `containsNull = false` or a `MapType` with `containsNull = false`. For example, in `ArrayType` case, current implementation leaves a cast `cast(value#63 as array<double>).toDoubleArray`. However, we can eliminate `cast(value#63 as array<double>)` if we know `value#63` does not include `null`. This PR apply this elimination for `ArrayType` and `MapType` in `SimplifyCasts` at a plan optimization phase. In summary, we got 1.2-1.3x performance improvements over the code before applying this PR. Here are performance results of benchmark programs: ``` test("Read array in Dataset") { import sparkSession.implicits._ val iters = 5 val n = 1024 * 1024 val rows = 15 val benchmark = new Benchmark("Read primnitive array", n) val rand = new Random(511) val intDS = sparkSession.sparkContext.parallelize(0 until rows, 1) .map(i => Array.tabulate(n)(i => i)).toDS() intDS.count() // force to create ds val lastElement = n - 1 val randElement = rand.nextInt(lastElement) benchmark.addCase(s"Read int array in Dataset", numIters = iters)(iter => { val idx0 = randElement val idx1 = lastElement intDS.map(a => a(0) + a(idx0) + a(idx1)).collect }) val doubleDS = sparkSession.sparkContext.parallelize(0 until rows, 1) .map(i => Array.tabulate(n)(i => i.toDouble)).toDS() doubleDS.count() // force to create ds benchmark.addCase(s"Read double array in Dataset", numIters = iters)(iter => { val idx0 = randElement val idx1 = lastElement doubleDS.map(a => a(0) + a(idx0) + a(idx1)).collect }) benchmark.run() } Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.4 Intel(R) Core(TM) i5-5257U CPU 2.70GHz without this PR Read primnitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Read int array in Dataset 525 / 690 2.0 500.9 1.0X Read double array in Dataset 947 / 1209 1.1 902.7 0.6X with this PR Read primnitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Read int array in Dataset 400 / 492 2.6 381.5 1.0X Read double array in Dataset 788 / 870 1.3 751.4 0.5X ``` An example program that originally caused this performance issue. ``` val ds = Seq(Array(1.0, 2.0, 3.0), Array(4.0, 5.0, 6.0)).toDS() val ds2 = ds.map(p => { var s = 0.0 for (i <- 0 to 2) { s += p(i) } s }) ds2.show ds2.explain(true) ``` Plans before this PR ``` == Parsed Logical Plan == 'SerializeFromObject [input[0, double, true] AS value#68] +- 'MapElements <function1>, obj#67: double +- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#66: [D +- LocalRelation [value#63] == Analyzed Logical Plan == value: double SerializeFromObject [input[0, double, true] AS value#68] +- MapElements <function1>, obj#67: double +- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D +- LocalRelation [value#63] == Optimized Logical Plan == SerializeFromObject [input[0, double, true] AS value#68] +- MapElements <function1>, obj#67: double +- DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D +- LocalRelation [value#63] == Physical Plan == *SerializeFromObject [input[0, double, true] AS value#68] +- *MapElements <function1>, obj#67: double +- *DeserializeToObject cast(value#63 as array<double>).toDoubleArray, obj#66: [D +- LocalTableScan [value#63] ``` Plans after this PR ``` == Parsed Logical Plan == 'SerializeFromObject [input[0, double, true] AS value#6] +- 'MapElements <function1>, obj#5: double +- 'DeserializeToObject unresolveddeserializer(upcast(getcolumnbyordinal(0, ArrayType(DoubleType,false)), ArrayType(DoubleType,false), - root class: "scala.Array").toDoubleArray), obj#4: [D +- LocalRelation [value#1] == Analyzed Logical Plan == value: double SerializeFromObject [input[0, double, true] AS value#6] +- MapElements <function1>, obj#5: double +- DeserializeToObject cast(value#1 as array<double>).toDoubleArray, obj#4: [D +- LocalRelation [value#1] == Optimized Logical Plan == SerializeFromObject [input[0, double, true] AS value#6] +- MapElements <function1>, obj#5: double +- DeserializeToObject value#1.toDoubleArray, obj#4: [D +- LocalRelation [value#1] == Physical Plan == *SerializeFromObject [input[0, double, true] AS value#6] +- *MapElements <function1>, obj#5: double +- *DeserializeToObject value#1.toDoubleArray, obj#4: [D +- LocalTableScan [value#1] ``` ## How was this patch tested? Tested by new test cases in `SimplifyCastsSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #13704 from kiszk/SPARK-15985.
* [SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with class ↵Shixiong Zhu2016-08-301-1/+1
| | | | | | | | | | | | | | | | | | defined in repl ## What changes were proposed in this pull request? There are a lot of failures recently: http://spark-tests.appspot.com/tests/org.apache.spark.repl.ReplSuite/replicating%20blocks%20of%20object%20with%20class%20defined%20in%20repl This PR just changed the persist level to `MEMORY_AND_DISK_2` to avoid blocks being evicted from memory. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14884 from zsxwing/SPARK-17318.
* [SPARK-17243][WEB UI] Spark 2.0 History Server won't load with very large ↵Alex Bozarth2016-08-308-7/+106
| | | | | | | | | | | | | | | | | | application history ## What changes were proposed in this pull request? With the new History Server the summary page loads the application list via the the REST API, this makes it very slow to impossible to load with large (10K+) application history. This pr fixes this by adding the `spark.history.ui.maxApplications` conf to limit the number of applications the History Server displays. This is accomplished using a new optional `limit` param for the `applications` api. (Note this only applies to what the summary page displays, all the Application UI's are still accessible if the user knows the App ID and goes to the Application UI directly.) I've also added a new test for the `limit` param in `HistoryServerSuite.scala` ## How was this patch tested? Manual testing and dev/run-tests Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #14835 from ajbozarth/spark17243.
* [SPARK-17314][CORE] Use Netty's DefaultThreadFactory to enable its fast ↵Shixiong Zhu2016-08-301-5/+2
| | | | | | | | | | | | | | | | ThreadLocal impl ## What changes were proposed in this pull request? When a thread is a Netty's FastThreadLocalThread, Netty will use its fast ThreadLocal implementation. It has a better performance than JDK's (See the benchmark results in https://github.com/netty/netty/pull/4417, note: it's not a fix to Netty's FastThreadLocal. It just fixed an issue in Netty's benchmark codes) This PR just changed the ThreadFactory to Netty's DefaultThreadFactory which will use FastThreadLocalThread. There is also a minor change to the thread names. See https://github.com/netty/netty/blob/netty-4.0.22.Final/common/src/main/java/io/netty/util/concurrent/DefaultThreadFactory.java#L94 ## How was this patch tested? Author: Shixiong Zhu <shixiong@databricks.com> Closes #14879 from zsxwing/netty-thread.
* [SPARK-17304] Fix perf. issue caused by ↵Josh Rosen2016-08-301-9/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | TaskSetManager.abortIfCompletelyBlacklisted This patch addresses a minor scheduler performance issue that was introduced in #13603. If you run ``` sc.parallelize(1 to 100000, 100000).map(identity).count() ``` then most of the time ends up being spent in `TaskSetManager.abortIfCompletelyBlacklisted()`: ![image](https://cloud.githubusercontent.com/assets/50748/18071032/428732b0-6e07-11e6-88b2-c9423cd61f53.png) When processing resource offers, the scheduler uses a nested loop which considers every task set at multiple locality levels: ```scala for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { do { launchedTask = resourceOfferSingleTaskSet( taskSet, maxLocality, shuffledOffers, availableCpus, tasks) } while (launchedTask) } ``` In order to prevent jobs with globally blacklisted tasks from hanging, #13603 added a `taskSet.abortIfCompletelyBlacklisted` call inside of `resourceOfferSingleTaskSet`; if a call to `resourceOfferSingleTaskSet` fails to schedule any tasks, then `abortIfCompletelyBlacklisted` checks whether the tasks are completely blacklisted in order to figure out whether they will ever be schedulable. The problem with this placement of the call is that the last call to `resourceOfferSingleTaskSet` in the `while` loop will return `false`, implying that `resourceOfferSingleTaskSet` will call `abortIfCompletelyBlacklisted`, so almost every call to `resourceOffers` will trigger the `abortIfCompletelyBlacklisted` check for every task set. Instead, I think that this call should be moved out of the innermost loop and should be called _at most_ once per task set in case none of the task set's tasks can be scheduled at any locality level. Before this patch's changes, the microbenchmark example that I posted above took 35 seconds to run, but it now only takes 15 seconds after this change. /cc squito and kayousterhout for review. Author: Josh Rosen <joshrosen@databricks.com> Closes #14871 from JoshRosen/bail-early-if-no-cpus.
* [SPARK-5682][CORE] Add encrypted shuffle in sparkFerdinand Xu2016-08-3027-28/+478
| | | | | | | | | This patch is using Apache Commons Crypto library to enable shuffle encryption support. Author: Ferdinand Xu <cheng.a.xu@intel.com> Author: kellyzly <kellyzly@126.com> Closes #8880 from winningsix/SPARK-10771.
* [MINOR][MLLIB][SQL] Clean up unused variables and unused importXin Ren2016-08-3010-14/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Clean up unused variables and unused import statements, unnecessary `return` and `toArray`, and some more style improvement, when I walk through the code examples. ## How was this patch tested? Testet manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14836 from keypointt/codeWalkThroughML.
* [MINOR][DOCS] Fix minor typos in python example codeDmitriy Sokolov2016-08-306-77/+77
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix minor typos python example code in streaming programming guide ## How was this patch tested? N/A Author: Dmitriy Sokolov <silentsokolov@gmail.com> Closes #14805 from silentsokolov/fix-typos.
* [SPARK-17264][SQL] DataStreamWriter should document that it only supports ↵Sean Owen2016-08-302-2/+2
| | | | | | | | | | | | | | | | Parquet for now ## What changes were proposed in this pull request? Clarify that only parquet files are supported by DataStreamWriter now ## How was this patch tested? (Doc build -- no functional changes to test) Author: Sean Owen <sowen@cloudera.com> Closes #14860 from srowen/SPARK-17264.
* [SPARK-17276][CORE][TEST] Stop env params output on Jenkins job pageXin Ren2016-08-302-3/+3
| | | | | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-17276 ## What changes were proposed in this pull request? When trying to find error msg in a failed Jenkins build job, I'm annoyed by the huge env output. The env parameter output should be muted. ![screen shot 2016-08-26 at 10 52 07 pm](https://cloud.githubusercontent.com/assets/3925641/18025581/b8d567ba-6be2-11e6-9eeb-6aec223f1730.png) ## How was this patch tested? Tested manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14848 from keypointt/SPARK-17276.
* [SPARK-17234][SQL] Table Existence Checking when Index Table with the Same ↵gatorsmile2016-08-306-1/+62
| | | | | | | | | | | | | | | | | | | Name Exists ### What changes were proposed in this pull request? Hive Index tables are not supported by Spark SQL. Thus, we issue an exception when users try to access Hive Index tables. When the internal function `tableExists` tries to access Hive Index tables, it always gets the same error message: ```Hive index table is not supported```. This message could be confusing to users, since their SQL operations could be completely unrelated to Hive Index tables. For example, when users try to alter a table to a new name and there exists an index table with the same name, the expected exception should be a `TableAlreadyExistsException`. This PR made the following changes: - Introduced a new `AnalysisException` type: `SQLFeatureNotSupportedException`. When users try to access an `Index Table`, we will issue a `SQLFeatureNotSupportedException`. - `tableExists` returns `true` when hitting a `SQLFeatureNotSupportedException` and the feature is `Hive index table`. - Add a checking `requireTableNotExists` for `SessionCatalog`'s `createTable` API; otherwise, the current implementation relies on the Hive's internal checking. ### How was this patch tested? Added a test case Author: gatorsmile <gatorsmile@gmail.com> Closes #14801 from gatorsmile/tableExists.
* [SPARK-17289][SQL] Fix a bug to satisfy sort requirements in partial ↵Takeshi YAMAMURO2016-08-302-2/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | aggregations ## What changes were proposed in this pull request? Partial aggregations are generated in `EnsureRequirements`, but the planner fails to check if partial aggregation satisfies sort requirements. For the following query: ``` val df2 = (0 to 1000).map(x => (x % 2, x.toString)).toDF("a", "b").createOrReplaceTempView("t2") spark.sql("select max(b) from t2 group by a").explain(true) ``` Now, the SortAggregator won't insert Sort operator before partial aggregation, this will break sort-based partial aggregation. ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- LocalTableScan [a#5, b#6] ``` Actually, a correct plan is: ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- *Sort [a#5 ASC], false, 0 +- LocalTableScan [a#5, b#6] ``` ## How was this patch tested? Added tests in `PlannerSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14865 from maropu/SPARK-17289.
* [SPARK-17303] Added spark-warehouse to dev/.rat-excludesfrreiss2016-08-291-0/+1
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Excludes the `spark-warehouse` directory from the Apache RAT checks that src/run-tests performs. `spark-warehouse` is created by some of the Spark SQL tests, as well as by `bin/spark-sql`. ## How was this patch tested? Ran src/run-tests twice. The second time, the script failed because the first iteration Made the change in this PR. Ran src/run-tests a third time; RAT checks succeeded. Author: frreiss <frreiss@us.ibm.com> Closes #14870 from frreiss/fred-17303.
* [SPARK-17301][SQL] Remove unused classTag field from AtomicType base classJosh Rosen2016-08-301-9/+1
| | | | | | | | There's an unused `classTag` val in the AtomicType base class which is causing unnecessary slowness in deserialization because it needs to grab ScalaReflectionLock and create a new runtime reflection mirror. Removing this unused code gives a small but measurable performance boost in SQL task deserialization. Author: Josh Rosen <joshrosen@databricks.com> Closes #14869 from JoshRosen/remove-unused-classtag.
* [SPARK-16581][SPARKR] Make JVM backend calling functions publicShivaram Venkataraman2016-08-294-2/+167
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change exposes a public API in SparkR to create objects, call methods on the Spark driver JVM ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Unit tests, CRAN checks Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #14775 from shivaram/sparkr-java-api.
* [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastoreDavies Liu2016-08-297-33/+200
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions). It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default). ## How was this patch tested? Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster). Author: Davies Liu <davies@databricks.com> Closes #14607 from davies/repair_batch.
* [SPARKR][MINOR] Fix LDA docJunyang Qian2016-08-291-3/+7
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR tries to fix the name of the `SparkDataFrame` used in the example. Also, it gives a reference url of an example data file so that users can play with. ## How was this patch tested? Manual test. Author: Junyang Qian <junyangq@databricks.com> Closes #14853 from junyangq/SPARKR-FixLDADoc.
* fixed a typoSeigneurin, Alexis (CONT)2016-08-291-1/+1
| | | | | | | | idempotant -> idempotent Author: Seigneurin, Alexis (CONT) <Alexis.Seigneurin@capitalone.com> Closes #14833 from aseigneurin/fix-typo.