- Jan 10, 2017
-
-
Sean Owen authored
## What changes were proposed in this pull request? Updates to libthrift 0.9.3 to address a CVE. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #16530 from srowen/SPARK-18997. (cherry picked from commit 856bae6a) Signed-off-by:
Marcelo Vanzin <vanzin@cloudera.com>
-
Shixiong Zhu authored
[SPARK-19113][SS][TESTS] Set UncaughtExceptionHandler in onQueryStarted to ensure catching fatal errors during query initialization ## What changes were proposed in this pull request? StreamTest sets `UncaughtExceptionHandler` after starting the query now. It may not be able to catch fatal errors during query initialization. This PR uses `onQueryStarted` callback to fix it. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16492 from zsxwing/SPARK-19113.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? `DataStreamReaderWriterSuite` makes test files in source folder like the followings. Interestingly, the root cause is `withSQLConf` fails to reset `OptionalConfigEntry` correctly. In other words, it resets the config into `Some(undefined)`. ```bash $ git status Untracked files: (use "git add <file>..." to include in what will be committed) sql/core/%253Cundefined%253E/ sql/core/%3Cundefined%3E/ ``` ## How was this patch tested? Manual. ``` build/sbt "project sql" test git status ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16522 from dongjoon-hyun/SPARK-19137. (cherry picked from commit d5b1dc93) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Liwei Lin authored
## What changes were proposed in this pull request? Prior to this patch, we'll generate `compare(...)` for `GeneratedClass$SpecificOrdering` like below, leading to Janino exceptions saying the code grows beyond 64 KB. ``` scala /* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering { /* ..... */ ... /* 10969 */ private int compare(InternalRow a, InternalRow b) { /* 10970 */ InternalRow i = null; // Holds current row being evaluated. /* 10971 */ /* 1.... */ code for comparing field0 /* 1.... */ code for comparing field1 /* 1.... */ ... /* 1.... */ code for comparing field449 /* 15012 */ /* 15013 */ return 0; /* 15014 */ } /* 15015 */ } ``` This patch would break `compare(...)` into smaller `compare_xxx(...)` methods when necessary; then we'll get generated `compare(...)` like: ``` scala /* 001 */ public SpecificOrdering generate(Object[] references) { /* 002 */ return new SpecificOrdering(references); /* 003 */ } /* 004 */ /* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering { /* 006 */ /* 007 */ ... /* 1.... */ /* 11290 */ private int compare_0(InternalRow a, InternalRow b) { /* 11291 */ InternalRow i = null; // Holds current row being evaluated. /* 11292 */ /* 11293 */ i = a; /* 11294 */ boolean isNullA; /* 11295 */ UTF8String primitiveA; /* 11296 */ { /* 11297 */ /* 11298 */ Object obj = ((Expression) references[0]).eval(null); /* 11299 */ UTF8String value = (UTF8String) obj; /* 11300 */ isNullA = false; /* 11301 */ primitiveA = value; /* 11302 */ } /* 11303 */ i = b; /* 11304 */ boolean isNullB; /* 11305 */ UTF8String primitiveB; /* 11306 */ { /* 11307 */ /* 11308 */ Object obj = ((Expression) references[0]).eval(null); /* 11309 */ UTF8String value = (UTF8String) obj; /* 11310 */ isNullB = false; /* 11311 */ primitiveB = value; /* 11312 */ } /* 11313 */ if (isNullA && isNullB) { /* 11314 */ // Nothing /* 11315 */ } else if (isNullA) { /* 11316 */ return -1; /* 11317 */ } else if (isNullB) { /* 11318 */ return 1; /* 11319 */ } else { /* 11320 */ int comp = primitiveA.compare(primitiveB); /* 11321 */ if (comp != 0) { /* 11322 */ return comp; /* 11323 */ } /* 11324 */ } /* 11325 */ /* 11326 */ /* 11327 */ i = a; /* 11328 */ boolean isNullA1; /* 11329 */ UTF8String primitiveA1; /* 11330 */ { /* 11331 */ /* 11332 */ Object obj1 = ((Expression) references[1]).eval(null); /* 11333 */ UTF8String value1 = (UTF8String) obj1; /* 11334 */ isNullA1 = false; /* 11335 */ primitiveA1 = value1; /* 11336 */ } /* 11337 */ i = b; /* 11338 */ boolean isNullB1; /* 11339 */ UTF8String primitiveB1; /* 11340 */ { /* 11341 */ /* 11342 */ Object obj1 = ((Expression) references[1]).eval(null); /* 11343 */ UTF8String value1 = (UTF8String) obj1; /* 11344 */ isNullB1 = false; /* 11345 */ primitiveB1 = value1; /* 11346 */ } /* 11347 */ if (isNullA1 && isNullB1) { /* 11348 */ // Nothing /* 11349 */ } else if (isNullA1) { /* 11350 */ return -1; /* 11351 */ } else if (isNullB1) { /* 11352 */ return 1; /* 11353 */ } else { /* 11354 */ int comp = primitiveA1.compare(primitiveB1); /* 11355 */ if (comp != 0) { /* 11356 */ return comp; /* 11357 */ } /* 11358 */ } /* 1.... */ /* 1.... */ ... /* 1.... */ /* 12652 */ return 0; /* 12653 */ } /* 1.... */ /* 1.... */ ... /* 15387 */ /* 15388 */ public int compare(InternalRow a, InternalRow b) { /* 15389 */ /* 15390 */ int comp_0 = compare_0(a, b); /* 15391 */ if (comp_0 != 0) { /* 15392 */ return comp_0; /* 15393 */ } /* 15394 */ /* 15395 */ int comp_1 = compare_1(a, b); /* 15396 */ if (comp_1 != 0) { /* 15397 */ return comp_1; /* 15398 */ } /* 1.... */ /* 1.... */ ... /* 1.... */ /* 15450 */ return 0; /* 15451 */ } /* 15452 */ } ``` ## How was this patch tested? - a new added test case which - would fail prior to this patch - would pass with this patch - ordering correctness should already be covered by existing tests like those in `OrderingSuite` ## Acknowledgement A major part of this PR - the refactoring work of `splitExpression()` - has been done by ueshin. Author: Liwei Lin <lwlin7@gmail.com> Author: Takuya UESHIN <ueshin@happy-camper.st> Author: Takuya Ueshin <ueshin@happy-camper.st> Closes #15480 from lw-lin/spec-ordering-64k-. (cherry picked from commit acfc5f35) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
- Jan 09, 2017
-
-
Burak Yavuz authored
## What changes were proposed in this pull request? Backport for #16361 to 2.1 branch. ## How was this patch tested? Unit tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #16518 from brkyvz/reg-break-2.1.
-
- Jan 08, 2017
-
-
Felix Cheung authored
## What changes were proposed in this pull request? backport to 2.1 Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16507 from felixcheung/portsparkuir21.
-
anabranch authored
## What changes were proposed in this pull request? - [X] Make sure all join types are clearly mentioned - [X] Make join labeling/style consistent - [X] Make join label ordering docs the same - [X] Improve join documentation according to above for Scala - [X] Improve join documentation according to above for Python - [X] Improve join documentation according to above for R ## How was this patch tested? No tests b/c docs. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Closes #16504 from anabranch/SPARK-19126. (cherry picked from commit 19d9d4c8) Signed-off-by:
Felix Cheung <felixcheung@apache.org>
-
anabranch authored
## What changes were proposed in this pull request? - [X] Fix inconsistencies in function reference for dense rank and dense - [X] Make all languages equivalent in their reference to `dense_rank` and `rank`. ## How was this patch tested? N/A for docs. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Closes #16505 from anabranch/SPARK-19127. (cherry picked from commit 1f6ded64) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
- Jan 07, 2017
-
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? This PR adds a new behavior change description on `CREATE TABLE ... LOCATION` at `sql-programming-guide.md` clearly under `Upgrading From Spark SQL 1.6 to 2.0`. This change is introduced at Apache Spark 2.0.0 as [SPARK-15276](https://issues.apache.org/jira/browse/SPARK-15276). ## How was this patch tested? ``` SKIP_API=1 jekyll build ``` **Newly Added Description** <img width="913" alt="new" src="https://cloud.githubusercontent.com/assets/9700541/21743606/7efe2b12-d4ba-11e6-8a0d-551222718ea2.png "> Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16400 from dongjoon-hyun/SPARK-18941. (cherry picked from commit 923e5948) Signed-off-by:
gatorsmile <gatorsmile@gmail.com>
-
Sean Owen authored
configuration.html section headings were not specified correctly in markdown and weren't rendering, being recognized correctly. Removed extra p tags and pulled level 4 titles up to level 3, since level 3 had been skipped. This improves the TOC. Doc build, manual check. Author: Sean Owen <sowen@cloudera.com> Closes #16490 from srowen/SPARK-19106. (cherry picked from commit 54138f6e) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
wm624@hotmail.com authored
[SPARK-19110][ML][MLLIB] DistributedLDAModel returns different logPrior for original and loaded model ## What changes were proposed in this pull request? While adding DistributedLDAModel training summary for SparkR, I found that the logPrior for original and loaded model is different. For example, in the test("read/write DistributedLDAModel"), I add the test: val logPrior = model.asInstanceOf[DistributedLDAModel].logPrior val logPrior2 = model2.asInstanceOf[DistributedLDAModel].logPrior assert(logPrior === logPrior2) The test fails: -4.394180878889078 did not equal -4.294290536919573 The reason is that `graph.vertices.aggregate(0.0)(seqOp, _ + _)` only returns the value of a single vertex instead of the aggregation of all vertices. Therefore, when the loaded model does the aggregation in a different order, it returns different `logPrior`. Please refer to #16464 for details. ## How was this patch tested? Add a new unit test for testing logPrior. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16491 from wangmiao1981/ldabug. (cherry picked from commit 036b5034) Signed-off-by:
Joseph K. Bradley <joseph@databricks.com>
-
- Jan 06, 2017
-
-
Tathagata Das authored
[SPARK-19074][SS][DOCS] Updated Structured Streaming Programming Guide for update mode and source/sink options ## What changes were proposed in this pull request? Updates - Updated Late Data Handling section by adding a figure for Update Mode. Its more intuitive to explain late data handling with Update Mode, so I added the new figure before the Append Mode figure. - Updated Output Modes section with Update mode - Added options for all the sources and sinks --------------------------- ---------------------------  --------------------------- --------------------------- <img width="931" alt="screen shot 2017-01-03 at 6 09 11 pm" src="https://cloud.githubusercontent.com/assets/663212/21629740/d21c9bb8-d1df-11e6-915b-488a59589fa6.png"> <img width="933" alt="screen shot 2017-01-03 at 6 10 00 pm" src="https://cloud.githubusercontent.com/assets/663212/21629749/e22bdabe-d1df-11e6-86d3-7e51d2f28dbc.png"> --------------------------- ---------------------------    Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16468 from tdas/SPARK-19074. (cherry picked from commit b59cddab) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
zuotingbing authored
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-19083# sbin/start-history-server.sh script use of $ without quotes, this will affect the length of args which used in HistoryServerArguments::parse(args: List[String]) Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #16484 from zuotingbing/sh. (cherry picked from commit a9a13737) Signed-off-by:
Marcelo Vanzin <vanzin@cloudera.com>
-
jerryshao authored
## What changes were proposed in this pull request? Current HistoryServer's ACLs is derived from application event-log, which means the newly changed ACLs cannot be applied to the old data, this will become a problem where newly added admin cannot access the old application history UI, only the new application can be affected. So here propose to add admin ACLs for history server, any configured user/group could have the view access to all the applications, while the view ACLs derived from application run-time still take effect. ## How was this patch tested? Unit test added. Author: jerryshao <sshao@hortonworks.com> Closes #16470 from jerryshao/SPARK-19033. (cherry picked from commit 4a4c3dc9) Signed-off-by:
Tom Graves <tgraves@yahoo-inc.com>
-
- Jan 04, 2017
-
-
Dongjoon Hyun authored
[SPARK-18877][SQL][BACKPORT-2.1] CSVInferSchema.inferField` on DecimalType should find a common type with `typeSoFar` ## What changes were proposed in this pull request? CSV type inferencing causes `IllegalArgumentException` on decimal numbers with heterogeneous precisions and scales because the current logic uses the last decimal type in a **partition**. Specifically, `inferRowType`, the **seqOp** of **aggregate**, returns the last decimal type. This PR fixes it to use `findTightestCommonType`. **decimal.csv** ``` 9.03E+12 1.19E+11 ``` **BEFORE** ```scala scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema root |-- _c0: decimal(3,-9) (nullable = true) scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show 16/12/16 14:32:49 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4) java.lang.IllegalArgumentException: requirement failed: Decimal precision 4 exceeds max precision 3 ``` **AFTER** ```scala scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").printSchema root |-- _c0: decimal(4,-9) (nullable = true) scala> spark.read.format("csv").option("inferSchema", true).load("decimal.csv").show +---------+ | _c0| +---------+ |9.030E+12| | 1.19E+11| +---------+ ``` ## How was this patch tested? Pass the newly add test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16463 from dongjoon-hyun/SPARK-18877-BACKPORT-21.
-
- Jan 03, 2017
-
-
gatorsmile authored
[SPARK-19048][SQL] Delete Partition Location when Dropping Managed Partitioned Tables in InMemoryCatalog ### What changes were proposed in this pull request? The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition. This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`. ### How was this patch tested? Added test cases for both HiveExternalCatalog and InMemoryCatalog Author: gatorsmile <gatorsmile@gmail.com> Closes #16448 from gatorsmile/unsetSerdeProp. (cherry picked from commit b67b35f7) Signed-off-by:
gatorsmile <gatorsmile@gmail.com>
-
- Jan 02, 2017
-
-
gatorsmile authored
### What changes were proposed in this pull request? Fixed non-thread-safe functions used in SessionCatalog: - refreshTable - lookupRelation ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #16437 from gatorsmile/addSyncToLookUpTable. (cherry picked from commit 35e97407) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Liwei Lin authored
## What changes were proposed in this pull request? Currently some code snippets in the programming guide just do not compile. We should fix them. ## How was this patch tested? ``` SKIP_API=1 jekyll build ``` ## Screenshot from part of the change:  Author: Liwei Lin <lwlin7@gmail.com> Closes #16442 from lw-lin/ss-pro-guide-.
-
genmao.ygm authored
## What changes were proposed in this pull request? The largest parallelism in PartitioningAwareFileIndex #listLeafFilesInParallel() is 10000 in hard code. We may need to make this number configurable. And in PR, I reduce it to 100. ## How was this patch tested? Existing ut. Author: genmao.ygm <genmao.ygm@genmaoygmdeMacBook-Air.local> Author: dylon <hustyugm@gmail.com> Closes #15829 from uncleGen/SPARK-18379. (cherry picked from commit 745ab8bc) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? The configuration `spark.yarn.security.tokens.{service}.enabled` is deprecated. Now we should use `spark.yarn.security.credentials.{service}.enabled`. Some places in the doc is not updated yet. ## How was this patch tested? N/A. Just doc change. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16444 from viirya/minor-credential-provider-doc. (cherry picked from commit 0ac2f1e7) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Jan 01, 2017
-
-
Shixiong Zhu authored
## What changes were proposed in this pull request? `monthsSinceEpoch` in this test is like `math.floor(num)`, so `monthDiff` has two possible values. ## How was this patch tested? Jenkins. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16449 from zsxwing/watermark-test-hotfix. (cherry picked from commit 23940473) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
- Dec 30, 2016
-
-
Cheng Lian authored
This PR documents the scalable partition handling feature in the body of the programming guide. Before this PR, we only mention it in the migration guide. It's not super clear that external datasource tables require an extra `MSCK REPAIR TABLE` command is to have per-partition information persisted since 2.1. N/A. Author: Cheng Lian <lian@databricks.com> Closes #16424 from liancheng/scalable-partition-handling-doc. (cherry picked from commit 871f6114) Signed-off-by:
Cheng Lian <lian@databricks.com>
-
- Dec 29, 2016
-
-
adesharatushar authored
[SPARK-19003][DOCS] Add Java example in Spark Streaming Guide, section Design Patterns for using foreachRDD ## What changes were proposed in this pull request? Added missing Java example under section "Design Patterns for using foreachRDD". Now this section has examples in all 3 languages, improving consistency of documentation. ## How was this patch tested? Manual. Generated docs using command "SKIP_API=1 jekyll build" and verified generated HTML page manually. The syntax of example has been tested for correctness using sample code on Java1.7 and Spark 2.2.0-SNAPSHOT. Author: adesharatushar <tushar_adeshara@persistent.com> Closes #16408 from adesharatushar/streaming-doc-fix. (cherry picked from commit dba81e1d) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Dec 28, 2016
-
-
Tathagata Das authored
[SPARK-18669][SS][DOCS] Update Apache docs for Structured Streaming regarding watermarking and status ## What changes were proposed in this pull request? - Extended the Window operation section with code snippet and explanation of watermarking - Extended the Output Mode section with a table showing the compatibility between query type and output mode - Rewrote the Monitoring section with updated jsons generated by StreamingQuery.progress/status - Updated API changes in the StreamingQueryListener example TODO - [x] Figure showing the watermarking ## How was this patch tested? N/A ## Screenshots ### Section: Windowed Aggregation with Event Time <img width="927" alt="screen shot 2016-12-15 at 3 33 10 pm" src="https://cloud.githubusercontent.com/assets/663212/21246197/0e02cb1a-c2dc-11e6-8816-0cd28d8201d7.png">  <img width="929" alt="screen shot 2016-12-15 at 3 33 46 pm" src="https://cloud.githubusercontent.com/assets/663212/21246202/1652cefa-c2dc-11e6-8c64-3c05977fb3fc.png"> ---------------------------- ### Section: Output Modes  ---------------------------- ### Section: Monitoring   Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16294 from tdas/SPARK-18669. (cherry picked from commit 092c6725) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Sean Owen authored
[SPARK-18993][BUILD] Unable to build/compile Spark in IntelliJ due to missing Scala deps in spark-tags ## What changes were proposed in this pull request? This adds back a direct dependency on Scala library classes from spark-tags because its Scala annotations need them. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16418 from srowen/SPARK-18993. (cherry picked from commit d7bce3bd) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
Carson Wang authored
## What changes were proposed in this pull request? Fix the document of `ForeachWriter` to use `writeStream` instead of `write` for a streaming dataset. ## How was this patch tested? Docs only. Author: Carson Wang <carson.wang@intel.com> Closes #16419 from carsonwang/FixDoc. (cherry picked from commit 2a5f52a7) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Dec 24, 2016
-
-
Kousuke Saruta authored
## What changes were proposed in this pull request? This issue was reported by wangyum. In the AllJobsPage, JobPage and StagePage, the description length was limited before like as follows.  But recently, the limitation seems to have been accidentally removed.  The cause is that some tables are no longer `sortable` class although they were, and `sortable` class does not only mark tables as sortable but also limited the width of their child `td` elements. The reason why now some tables are not `sortable` class is because another sortable mechanism was introduced by #13620 and #13708 with pagination feature. To fix this issue, I've introduced new class `table-cell-width-limited` which limits the description cell width and the description is like what it was. <img width="1260" alt="2016-12-20 1 00 34" src="https://cloud.githubusercontent.com/assets/4736016/21320478/89141c7a-c654-11e6-8494-f8f91325980b.png "> ## How was this patch tested? Tested manually with my browser. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #16338 from sarutak/SPARK-18837. (cherry picked from commit f2ceb2ab) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Dec 23, 2016
-
-
Shixiong Zhu authored
[SPARK-18991][CORE] Change ContextCleaner.referenceBuffer to use ConcurrentHashMap to make it faster ## What changes were proposed in this pull request? The time complexity of ConcurrentHashMap's `remove` is O(1). Changing ContextCleaner.referenceBuffer's type from `ConcurrentLinkedQueue` to `ConcurrentHashMap's` will make the removal much faster. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16390 from zsxwing/SPARK-18991. (cherry picked from commit a848f0ba) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
- Dec 22, 2016
-
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Right now the name of threads created by Netty for Spark RPC are `shuffle-client-**` and `shuffle-server-**`. It's pretty confusing. This PR just uses the module name in TransportConf to set the thread name. In addition, it also includes the following minor fixes: - TransportChannelHandler.channelActive and channelInactive should call the corresponding super methods. - Make ShuffleBlockFetcherIterator throw NoSuchElementException if it has no more elements. Otherwise, if the caller calls `next` without `hasNext`, it will just hang. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16380 from zsxwing/SPARK-18972. (cherry picked from commit f252cb5d) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Add missing InterfaceStability.Evolving for Structured Streaming APIs ## How was this patch tested? Compiling the codes. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16385 from zsxwing/SPARK-18985. (cherry picked from commit 2246ce88) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Ryan Williams authored
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR. Alternative to #16303. Author: Ryan Williams <ryan.blake.williams@gmail.com> Closes #16311 from ryan-williams/tt. (cherry picked from commit afd9bc1d) Signed-off-by:
Marcelo Vanzin <vanzin@cloudera.com>
-
Reynold Xin authored
## What changes were proposed in this pull request? SortPartitions and RedistributeData logical operators are not actually used and can be removed. Note that we do have a Sort operator (with global flag false) that subsumed SortPartitions. ## How was this patch tested? Also updated test cases to reflect the removal. Author: Reynold Xin <rxin@databricks.com> Closes #16381 from rxin/SPARK-18973. (cherry picked from commit 26151000) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
Reynold Xin authored
## What changes were proposed in this pull request? Starting Spark 2.1.0, bucketing feature is available for all file-based data sources. This patch fixes some function docs that haven't yet been updated to reflect that. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #16349 from rxin/ds-doc. (cherry picked from commit 2e861df9) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Reynold Xin authored
This patch includes minor changes to improve readability for partition handling code. I'm in the middle of implementing some new feature and found some naming / implicit type inference not as intuitive. This patch should have no semantic change and the changes should be covered by existing test cases. Author: Reynold Xin <rxin@databricks.com> Closes #16378 from rxin/minor-fix. (cherry picked from commit 7c5b7b3a) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? This PR audits places using `logicalPlan` in StreamExecution and ensures they all handles the case that `logicalPlan` cannot be created. In addition, this PR also fixes the following issues in `StreamingQueryException`: - `StreamingQueryException` and `StreamExecution` are cycle-dependent because in the `StreamingQueryException`'s constructor, it calls `StreamExecution`'s `toDebugString` which uses `StreamingQueryException`. Hence it will output `null` value in the error message. - Duplicated stack trace when calling Throwable.printStackTrace because StreamingQueryException's toString contains the stack trace. ## How was this patch tested? The updated `test("max files per trigger - incorrect values")`. I found this issue when I switched from `testStream` to the real codes to verify the failure in this test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16322 from zsxwing/SPARK-18907. (cherry picked from commit ff7d82a2) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
- Dec 21, 2016
-
-
Burak Yavuz authored
## What changes were proposed in this pull request? https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.streaming.InputStreamsSuite&test_name=socket+input+stream ## How was this patch tested? Tested 2,000 times. Author: Burak Yavuz <brkyvz@gmail.com> Closes #16343 from brkyvz/sock. (cherry picked from commit afe36516) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Takeshi YAMAMURO authored
## What changes were proposed in this pull request? This pr is to fix an `NullPointerException` issue caused by a following `limit + aggregate` query; ``` scala> val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value") scala> df.limit(2).groupBy("id").count().show WARN TaskSetManager: Lost task 0.0 in stage 9.0 (TID 8204, lvsp20hdn012.stubprod.com): java.lang.NullPointerException at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithKeys$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) ``` The root culprit is that [`$doAgg()`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L596) skips an initialization of [the buffer iterator](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L603); `BaseLimitExec` sets `stopEarly=true` and `$doAgg()` exits in the middle without the initialization. ## How was this patch tested? Added a test to check if no exception happens for limit + aggregates in `DataFrameAggregateSuite.scala`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #15980 from maropu/SPARK-18528. (cherry picked from commit b41ec997) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
Tathagata Das authored
## What changes were proposed in this pull request? Made update mode public. As part of that here are the changes. - Update DatastreamWriter to accept "update" - Changed package of InternalOutputModes from o.a.s.sql to o.a.s.sql.catalyst - Added update mode state removing with watermark to StateStoreSaveExec ## How was this patch tested? Added new tests in changed modules Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16360 from tdas/SPARK-18234. (cherry picked from commit 83a6ace0) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? When KafkaSource fails on Kafka errors, we should create a new consumer to retry rather than using the existing broken one because it's possible that the broken one will fail again. This PR also assigns a new group id to the new created consumer for a possible race condition: the broken consumer cannot talk with the Kafka cluster in `close` but the new consumer can talk to Kafka cluster. I'm not sure if this will happen or not. Just for safety to avoid that the Kafka cluster thinks there are two consumers with the same group id in a short time window. (Note: CachedKafkaConsumer doesn't need this fix since `assign` never uses the group id.) ## How was this patch tested? In https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70370/console , it ran this flaky test 120 times and all passed. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16282 from zsxwing/kafka-fix. (cherry picked from commit 95efc895) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
gatorsmile authored
### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/16356 to Spark 2.1.1 branch. ---- Currently, we only have a SQL interface for recovering all the partitions in the directory of a table and update the catalog. `MSCK REPAIR TABLE` or `ALTER TABLE table RECOVER PARTITIONS`. (Actually, very hard for me to remember `MSCK` and have no clue what it means) After the new "Scalable Partition Handling", the table repair becomes much more important for making visible the data in the created data source partitioned table. Thus, this PR is to add it into the Catalog interface. After this PR, users can repair the table by ```Scala spark.catalog.recoverPartitions("testTable") ``` ### How was this patch tested? Modified the existing test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #16372 from gatorsmile/repairTable2.1.1.
-