- Dec 21, 2016
-
-
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.
-
Shixiong Zhu authored
[SPARK-18954][TESTS] Fix flaky test: o.a.s.streaming.BasicOperationsSuite rdd cleanup - map and window ## What changes were proposed in this pull request? The issue in this test is the cleanup of RDDs may not be able to finish before stopping StreamingContext. This PR basically just puts the assertions into `eventually` and runs it before stopping StreamingContext. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16362 from zsxwing/SPARK-18954. (cherry picked from commit 078c71c2) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? The failure is because in `test("basic functionality")`, it doesn't block until `ExecutorAllocationManager.manageAllocation` is called. This PR just adds StreamManualClock to allow the tests to block on expected wait time to make the test deterministic. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16321 from zsxwing/SPARK-18031. (cherry picked from commit ccfe60a8) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Tathagata Das authored
## What changes were proposed in this pull request? Two changes - Fix how delays specified in months and years are translated to milliseconds - Following up on #16258, not show watermark when there is no watermarking in the query ## How was this patch tested? Updated and new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16304 from tdas/SPARK-18834-1. (cherry picked from commit 607a1e63) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Wenchen Fan authored
## What changes were proposed in this pull request? It's a huge waste to call `Catalog.listTables` in `SQLContext.tableNames`, which only need the table names, while `Catalog.listTables` will get the table metadata for each table name. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16352 from cloud-fan/minor. (cherry picked from commit b7650f11) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
- Dec 20, 2016
-
-
Burak Yavuz authored
## What changes were proposed in this pull request? It was pretty flaky before 10 days ago. https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.execution.streaming.state.StateStoreSuite&test_name=maintenance Since no code changes went into this code path to not be so flaky, I'm just increasing the timeouts such that load related flakiness shouldn't be a problem. As you may see from the testing, I haven't been able to reproduce it. ## How was this patch tested? 2000 retries 5 times Author: Burak Yavuz <brkyvz@gmail.com> Closes #16314 from brkyvz/maint-flaky. (cherry picked from commit b2dd8ec6) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Burak Yavuz authored
[SPARK-18927][SS] MemorySink for StructuredStreaming can't recover from checkpoint if location is provided in SessionConf ## What changes were proposed in this pull request? Checkpoint Location can be defined for a StructuredStreaming on a per-query basis by the `DataStreamWriter` options, but it can also be provided through SparkSession configurations. It should be able to recover in both cases when the OutputMode is Complete for MemorySinks. ## How was this patch tested? Unit tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #16342 from brkyvz/chk-rec. (cherry picked from commit caed8932) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? There is a timeout failure when using `rdd.toLocalIterator()` or `df.toLocalIterator()` for a PySpark RDD and DataFrame: df = spark.createDataFrame([[1],[2],[3]]) it = df.toLocalIterator() row = next(it) df2 = df.repartition(1000) # create many empty partitions which increase materialization time so causing timeout it2 = df2.toLocalIterator() row = next(it2) The cause of this issue is, we open a socket to serve the data from JVM side. We set timeout for connection and reading through the socket in Python side. In Python we use a generator to read the data, so we only begin to connect the socket once we start to ask data from it. If we don't consume it immediately, there is connection timeout. In the other side, the materialization time for RDD partitions is unpredictable. So we can't set a timeout for reading data through the socket. Otherwise, it is very possibly to fail. ## How was this patch tested? Added tests into PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16263 from viirya/fix-pyspark-localiterator. (cherry picked from commit 95c95b71) Signed-off-by:
Davies Liu <davies.liu@gmail.com>
-
Josh Rosen authored
## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
-
- Dec 19, 2016
-
-
Josh Rosen authored
## What changes were proposed in this pull request? In order to respond to task cancellation, Spark tasks must periodically check `TaskContext.isInterrupted()`, but this check is missing on a few critical read paths used in Spark SQL, including `FileScanRDD`, `JDBCRDD`, and UnsafeSorter-based sorts. This can cause interrupted / cancelled tasks to continue running and become zombies (as also described in #16189). This patch aims to fix this problem by adding `TaskContext.isInterrupted()` checks to these paths. Note that I could have used `InterruptibleIterator` to simply wrap a bunch of iterators but in some cases this would have an adverse performance penalty or might not be effective due to certain special uses of Iterators in Spark SQL. Instead, I inlined `InterruptibleIterator`-style logic into existing iterator subclasses. ## How was this patch tested? Tested manually in `spark-shell` with two different reproductions of non-cancellable tasks, one involving scans of huge files and another involving sort-merge joins that spill to disk. Both causes of zombie tasks are fixed by the changes added here. Author: Josh Rosen <joshrosen@databricks.com> Closes #16340 from JoshRosen/sql-task-interruption. (cherry picked from commit 5857b9ac) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
Wenchen Fan authored
## What changes were proposed in this pull request? It's weird that we use `Hive.getDatabase` to check the existence of a database, while Hive has a `databaseExists` interface. What's worse, `Hive.getDatabase` will produce an error message if the database doesn't exist, which is annoying when we only want to check the database existence. This PR fixes this and use `Hive.databaseExists` to check database existence. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16332 from cloud-fan/minor. (cherry picked from commit 7a75ee1c) Signed-off-by:
Yin Huai <yhuai@databricks.com>
-
xuanyuanking authored
## What changes were proposed in this pull request? As the scenario describe in [SPARK-18700](https://issues.apache.org/jira/browse/SPARK-18700 ), when cachedDataSourceTables invalided, the coming few queries will fetch all FileStatus in listLeafFiles function. In the condition of table has many partitions, these jobs will occupy much memory of driver finally may cause driver OOM. In this patch, add StripedLock for each table's relation in cache not for the whole cachedDataSourceTables, each table's load cache operation protected by it. ## How was this patch tested? Add a multi-thread access table test in `PartitionedTablePerfStatsSuite` and check it only loading once using metrics in `HiveCatalogMetrics` Author: xuanyuanking <xyliyuanjian@gmail.com> Closes #16135 from xuanyuanking/SPARK-18700. (cherry picked from commit 24482858) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
- Dec 18, 2016
-
-
gatorsmile authored
[SPARK-18703][SPARK-18675][SQL][BACKPORT-2.1] CTAS for hive serde table should work for all hive versions AND Drop Staging Directories and Data Files ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/16104 and https://github.com/apache/spark/pull/16134. ---------- [[SPARK-18675][SQL] CTAS for hive serde table should work for all hive versions](https://github.com/apache/spark/pull/16104) Before hive 1.1, when inserting into a table, hive will create the staging directory under a common scratch directory. After the writing is finished, hive will simply empty the table directory and move the staging directory to it. After hive 1.1, hive will create the staging directory under the table directory, and when moving staging directory to table directory, hive will still empty the table directory, but will exclude the staging directory there. In `InsertIntoHiveTable`, we simply copy the code from hive 1.2, which means we will always create the staging directory under the table directory, no matter what the hive version is. This causes problems if the hive version is prior to 1.1, because the staging directory will be removed by hive when hive is trying to empty the table directory. This PR copies the code from hive 0.13, so that we have 2 branches to create staging directory. If hive version is prior to 1.1, we'll go to the old style branch(i.e. create the staging directory under a common scratch directory), else, go to the new style branch(i.e. create the staging directory under the table directory) ---------- [[SPARK-18703] [SQL] Drop Staging Directories and Data Files After each Insertion/CTAS of Hive serde Tables](https://github.com/apache/spark/pull/16134) Below are the files/directories generated for three inserts againsts a Hive table: ``` /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/._SUCCESS.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/.part-00000.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/_SUCCESS /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/part-00000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/._SUCCESS.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/.part-00000.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/_SUCCESS /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/part-00000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/._SUCCESS.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/.part-00000.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/_SUCCESS /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/part-00000 /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.part-00000.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/part-00000 ``` The first 18 files are temporary. We do not drop it until the end of JVM termination. If JVM does not appropriately terminate, these temporary files/directories will not be dropped. Only the last two files are needed, as shown below. ``` /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.part-00000.crc /private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/part-00000 ``` The temporary files/directories could accumulate a lot when we issue many inserts, since each insert generats at least six files. This could eat a lot of spaces and slow down the JVM termination. When the JVM does not terminates approprately, the files might not be dropped. This PR is to drop the created staging files and temporary data files after each insert/CTAS. ### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #16325 from gatorsmile/backport-18703&18675.
-
Yuming Wang authored
## What changes were proposed in this pull request? `NoSuchElementException` will throw since https://github.com/apache/spark/pull/15056 if a broadcast cannot cache in memory. The reason is that that change cannot cover `!unrolled.hasNext` in `next()` function. This change is to cover the `!unrolled.hasNext` and check `hasNext` before calling `next` in `blockManager.getLocalValues` to make it more robust. We can cache and read broadcast even it cannot fit in memory from this pull request. Exception log: ``` 16/12/10 10:10:04 INFO UnifiedMemoryManager: Will not store broadcast_131 as the required space (1048576 bytes) exceeds our memory limit (122764 bytes) 16/12/10 10:10:04 WARN MemoryStore: Failed to reserve initial memory threshold of 1024.0 KB for computing block broadcast_131 in memory. 16/12/10 10:10:04 WARN MemoryStore: Not enough space to cache broadcast_131 in memory! (computed 384.0 B so far) 16/12/10 10:10:04 INFO MemoryStore: Memory use = 95.6 KB (blocks) + 0.0 B (scratch space shared across 0 tasks(s)) = 95.6 KB. Storage limit = 119.9 KB. 16/12/10 10:10:04 ERROR Utils: Exception encountered java.util.NoSuchElementException at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700) at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at scala.Option.map(Option.scala:146) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210) at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206) at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 16/12/10 10:10:04 ERROR Executor: Exception in task 1.0 in stage 86.0 (TID 134423) java.io.IOException: java.util.NoSuchElementException at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1276) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206) at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.util.NoSuchElementException at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700) at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at scala.Option.map(Option.scala:146) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210) at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269) ... 12 more ``` ## How was this patch tested? Add unit test Author: Yuming Wang <wgyumg@gmail.com> Closes #16252 from wangyum/SPARK-18827. (cherry picked from commit 1e5c51f3) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
gatorsmile authored
### What changes were proposed in this pull request? The configuration page looks messy now, as shown in the nightly build: https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/configuration.html Starting from the following location:  ### How was this patch tested? Attached is the screenshot generated in my local computer after the fix. [Configuration - Spark 2.2.0 Documentation.pdf](https://github.com/apache/spark/files/659315/Configuration.-.Spark.2.2.0.Documentation.pdf ) Author: gatorsmile <gatorsmile@gmail.com> Closes #16327 from gatorsmile/docFix. (cherry picked from commit c0c9e1d2) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Dec 17, 2016
-
-
Felix Cheung authored
## What changes were proposed in this pull request? Reorganizing content (copy/paste) ## How was this patch tested? https://felixcheung.github.io/sparkr-vignettes.html Previous: https://felixcheung.github.io/sparkr-vignettes_old.html Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16301 from felixcheung/rvignettespass2. (cherry picked from commit 38fd163d) Signed-off-by:
Felix Cheung <felixcheung@apache.org>
-
- Dec 16, 2016
-
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Merge two FileStreamSourceSuite files into one file. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16315 from zsxwing/FileStreamSourceSuite. (cherry picked from commit 4faa8a3e) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? SparkR tests, `R/run-tests.sh`, succeeds only once because `test_sparkSQL.R` does not clean up the test table, `people`. As a result, the rows in `people` table are accumulated at every run and the test cases fail. The following is the failure result for the second run. ```r Failed ------------------------------------------------------------------------- 1. Failure: create DataFrame from RDD (test_sparkSQL.R#204) ------------------- collect(sql("SELECT age from people WHERE name = 'Bob'"))$age not equal to c(16). Lengths differ: 2 vs 1 2. Failure: create DataFrame from RDD (test_sparkSQL.R#206) ------------------- collect(sql("SELECT height from people WHERE name ='Bob'"))$height not equal to c(176.5). Lengths differ: 2 vs 1 ``` ## How was this patch tested? Manual. Run `run-tests.sh` twice and check if it passes without failures. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16310 from dongjoon-hyun/SPARK-18897. (cherry picked from commit 1169db44) Signed-off-by:
Shivaram Venkataraman <shivaram@cs.berkeley.edu>
-
Takeshi YAMAMURO authored
## What changes were proposed in this pull request? A vectorized parquet reader fails to read column data if data schema and partition schema overlap with each other and inferred types in the partition schema differ from ones in the data schema. An example code to reproduce this bug is as follows; ``` scala> case class A(a: Long, b: Int) scala> val as = Seq(A(1, 2)) scala> spark.createDataFrame(as).write.parquet("/data/a=1/") scala> val df = spark.read.parquet("/data/") scala> df.printSchema root |-- a: long (nullable = true) |-- b: integer (nullable = true) scala> df.collect java.lang.NullPointerException at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.getLong(OnHeapColumnVector.java:283) at org.apache.spark.sql.execution.vectorized.ColumnarBatch$Row.getLong(ColumnarBatch.java:191) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at scala.collection.Iterator$$anon$11.next(Iterator.scala:409) at scala.collection.Iterator$$anon$11.next(Iterator.scala:409) ``` The root cause is that a logical layer (`HadoopFsRelation`) and a physical layer (`VectorizedParquetRecordReader`) have a different assumption on partition schema; the logical layer trusts the data schema to infer the type the overlapped partition columns, and, on the other hand, the physical layer trusts partition schema which is inferred from path string. To fix this bug, this pr simply updates `HadoopFsRelation.schema` to respect the partition columns position in data schema and respect the partition columns type in partition schema. ## How was this patch tested? Add tests in `ParquetPartitionDiscoverySuite` Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16030 from maropu/SPARK-18108. (cherry picked from commit dc2a4d4a) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? This PR adds StreamingQueryWrapper to make StreamExecution and progress classes serializable because it is too easy for it to get captured with normal usage. If StreamingQueryWrapper gets captured in a closure but no place calls its methods, it should not fail the Spark tasks. However if its methods are called, then this PR will throw a better message. ## How was this patch tested? `test("StreamingQuery should be Serializable but cannot be used in executors")` `test("progress classes should be Serializable")` Author: Shixiong Zhu <shixiong@databricks.com> Closes #16272 from zsxwing/SPARK-18850. (cherry picked from commit d7f3058e) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
- Dec 15, 2016
-
-
Reynold Xin authored
## What changes were proposed in this pull request? percentile_approx is the name used in Hive, and approx_percentile is the name used in Presto. approx_percentile is actually more consistent with our approx_count_distinct. Given the cost to alias SQL functions is low (one-liner), it'd be better to just alias them so it is easier to use. ## How was this patch tested? Technically I could add an end-to-end test to verify this one-line change, but it seemed too trivial to me. Author: Reynold Xin <rxin@databricks.com> Closes #16300 from rxin/SPARK-18892. (cherry picked from commit 172a52f5) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Patrick Wendell authored
-
Shivaram Venkataraman authored
Follow up to https://github.com/apache/spark/commit/ae853e8f3bdbd16427e6f1ffade4f63abaf74abb as `mv` throws an error on the Jenkins machines if source and destinations are the same. Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #16302 from shivaram/sparkr-no-mv-fix. (cherry picked from commit 5a44f18a) Signed-off-by:
Shivaram Venkataraman <shivaram@cs.berkeley.edu>
-
Patrick Wendell authored
-
Patrick Wendell authored
-
Shivaram Venkataraman authored
## What changes were proposed in this pull request? For release builds the R_PACKAGE_VERSION and VERSION are the same (e.g., 2.1.0). Thus `cp` throws an error which causes the build to fail. ## How was this patch tested? Manually by executing the following script ``` set -o pipefail set -e set -x touch a R_PACKAGE_VERSION=2.1.0 VERSION=2.1.0 if [ "$R_PACKAGE_VERSION" != "$VERSION" ]; then cp a a fi ``` Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #16299 from shivaram/sparkr-cp-fix. (cherry picked from commit 9634018c) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Burak Yavuz authored
## What changes were proposed in this pull request? Use `recentProgress` instead of `lastProgress` and filter out last non-zero value. Also add eventually to the latest assertQuery similar to first `assertQuery` ## How was this patch tested? Ran test 1000 times Author: Burak Yavuz <brkyvz@gmail.com> Closes #16287 from brkyvz/SPARK-18868. (cherry picked from commit 9c7f83b0) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Patrick Wendell authored
-
Patrick Wendell authored
-
Burak Yavuz authored
## What changes were proposed in this pull request? `_to_seq` wasn't imported. ## How was this patch tested? Added partitionBy to existing write path unit test Author: Burak Yavuz <brkyvz@gmail.com> Closes #16297 from brkyvz/SPARK-18888.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? When starting a stream with a lot of backfill and maxFilesPerTrigger, the user could often want to start with most recent files first. This would let you keep low latency for recent data and slowly backfill historical data. This PR adds a new option `latestFirst` to control this behavior. When it's true, `FileStreamSource` will sort the files by the modified time from latest to oldest, and take the first `maxFilesPerTrigger` files as a new batch. ## How was this patch tested? The added test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16251 from zsxwing/newest-first. (cherry picked from commit 68a6dc97) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Tathagata Das authored
## What changes were proposed in this pull request? Check whether Aggregation operators on a streaming subplan have aggregate expressions with isDistinct = true. ## How was this patch tested? Added unit test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16289 from tdas/SPARK-18870. (cherry picked from commit 4f7292c8) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
- Dec 14, 2016
-
-
Felix Cheung authored
## What changes were proposed in this pull request? doc cleanup ## How was this patch tested? ~~vignettes is not building for me. I'm going to kick off a full clean build and try again and attach output here for review.~~ Output html here: https://felixcheung.github.io/sparkr-vignettes.html Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16286 from felixcheung/rvignettespass. (cherry picked from commit 7d858bc5) Signed-off-by:
Shivaram Venkataraman <shivaram@cs.berkeley.edu>
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Since Apache Spark 1.4.0, R API document page has a broken link on `DESCRIPTION file` because Jekyll plugin script doesn't copy the file. This PR aims to fix that. - Official Latest Website: http://spark.apache.org/docs/latest/api/R/index.html - Apache Spark 2.1.0-rc2: http://people.apache.org/~pwendell/spark-releases/spark-2.1.0-rc2-docs/api/R/index.html ## How was this patch tested? Manual. ```bash cd docs SKIP_SCALADOC=1 jekyll build ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16292 from dongjoon-hyun/SPARK-18875. (cherry picked from commit ec0eae48) Signed-off-by:
Shivaram Venkataraman <shivaram@cs.berkeley.edu>
-
Reynold Xin authored
## What changes were proposed in this pull request? After the bug fix in SPARK-18854, TreeNode.apply now returns TreeNode[_] rather than a more specific type. It would be easier for interactive debugging to introduce a function that returns the BaseType. ## How was this patch tested? N/A - this is a developer only feature used for interactive debugging. As long as it compiles, it should be good to go. I tested this in spark-shell. Author: Reynold Xin <rxin@databricks.com> Closes #16288 from rxin/SPARK-18869. (cherry picked from commit 5d510c69) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Wenchen Fan authored
## What changes were proposed in this pull request? In `DataSource`, if the table is not analyzed, we will use 0 as the default value for table size. This is dangerous, we may broadcast a large table and cause OOM. We should use `defaultSizeInBytes` instead. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16280 from cloud-fan/bug. (cherry picked from commit d6f11a12) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
wm624@hotmail.com authored
## What changes were proposed in this pull request? When do the QA work, I found that the following issues: 1). `spark.mlp` doesn't include an example; 2). `spark.mlp` and `spark.lda` have redundant parameter explanations; 3). `spark.lda` document misses default values for some parameters. I also changed the `spark.logit` regParam in the examples, as we discussed in #16222. ## How was this patch tested? Manual test Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16284 from wangmiao1981/ks. (cherry picked from commit 32438853) Signed-off-by:
Felix Cheung <felixcheung@apache.org>
-
Reynold Xin authored
## What changes were proposed in this pull request? This is a bug introduced by subquery handling. numberedTreeString (which uses generateTreeString under the hood) numbers trees including innerChildren (used to print subqueries), but apply (which uses getNodeNumbered) ignores innerChildren. As a result, apply(i) would return the wrong plan node if there are subqueries. This patch fixes the bug. ## How was this patch tested? Added a test case in SubquerySuite.scala to test both the depth-first traversal of numbering as well as making sure the two methods are consistent. Author: Reynold Xin <rxin@databricks.com> Closes #16277 from rxin/SPARK-18854. (cherry picked from commit ffdd1fcd) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-