spark git commit: [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf under functions

2016-05-10 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 82f69594f -> 5a4a188fe [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf under functions ## What changes were proposed in this pull request? PR fixes the import issue which breaks udf functions. The following

spark git commit: [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf under functions

2016-05-10 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 93353b011 -> 89f73f674 [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf under functions ## What changes were proposed in this pull request? PR fixes the import issue which breaks udf functions. The following code

spark git commit: [SPARK-15131][SQL] Shutdown StateStore management thread when SparkContext has been shutdown

2016-05-04 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master ef55e46c9 -> bde27b89a [SPARK-15131][SQL] Shutdown StateStore management thread when SparkContext has been shutdown ## What changes were proposed in this pull request? Make sure that whenever the StateStoreCoordinator cannot be

spark git commit: [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock`

2016-05-04 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 ae79032dc -> 343c28504 [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock` ## What changes were proposed in this pull request? Currently in

spark git commit: [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock`

2016-05-04 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master a45647746 -> e597ec6f1 [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `ManualClock` ## What changes were proposed in this pull request? Currently in

spark git commit: [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master f5623b460 -> 659f635d3 [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption ## What changes were proposed in this pull request? Resetting the task interruption status before updating the task status. ## How was this

spark git commit: [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 50adfcdce -> 4bacdebe9 [SPARK-14234][CORE] Executor crashes for TaskRunner thread interruption ## What changes were proposed in this pull request? Resetting the task interruption status before updating the task status. ## How was this

spark git commit: [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 84b3a4a87 -> f5623b460 [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock ## What changes were proposed in this pull request? In some cases, fine-grained lock have race condition with class-loader

spark git commit: [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 a08d2c954 -> 50adfcdce [SPARK-15059][CORE] Remove fine-grained lock in ChildFirstURLClassLoader to avoid dead lock ## What changes were proposed in this pull request? In some cases, fine-grained lock have race condition with

spark git commit: [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 2b715251d -> 69e030226 [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow - that reduceFunc and invReduceFunc should be associative - that the intermediate result in iterated

spark git commit: [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master ca813330c -> 439e36101 [SPARK-9819][STREAMING][DOCUMENTATION] Clarify doc for invReduceFunc in incremental versions of reduceByWindow - that reduceFunc and invReduceFunc should be associative - that the intermediate result in iterated

spark git commit: [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-2.0 17996e7d0 -> 45bc65519 [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries ## What changes were proposed in this pull request? Since we've been processing continuous queries in separate threads, the call sites

spark git commit: [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries

2016-05-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 5503e453b -> 5bd9a2f69 [SPARK-14884][SQL][STREAMING][WEBUI] Fix call site for continuous queries ## What changes were proposed in this pull request? Since we've been processing continuous queries in separate threads, the call sites are

spark git commit: [SPARK-14833][SQL][STREAMING][TEST] Refactor StreamTests to test for source fault-tolerance correctly.

2016-04-23 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master ba5e0b87a -> 285385965 [SPARK-14833][SQL][STREAMING][TEST] Refactor StreamTests to test for source fault-tolerance correctly. ## What changes were proposed in this pull request? Current StreamTest allows testing of a streaming Dataset

spark git commit: [SPARK-14832][SQL][STREAMING] Refactor DataSource to ensure schema is inferred only once when creating a file stream

2016-04-22 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master c25b97fcc -> c431a76d0 [SPARK-14832][SQL][STREAMING] Refactor DataSource to ensure schema is inferred only once when creating a file stream ## What changes were proposed in this pull request? When creating a file stream using

spark git commit: [SPARK-14701][STREAMING] First stop the event loop, then stop the checkpoint writer in JobGenerator

2016-04-22 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 3647120a5 -> fde1340c7 [SPARK-14701][STREAMING] First stop the event loop, then stop the checkpoint writer in JobGenerator Currently if we call `streamingContext.stop` (e.g. in a `StreamingListener.onBatchCompleted` callback) when a

spark git commit: [SPARK-14699][CORE] Stop endpoints before closing the connections and don't stop client in Outbox

2016-04-21 Thread zsxwing
box should not close the client since it will be reused by others. This PR fixes it as well. ## How was this patch tested? test("SPARK-14699: RpcEnv.shutdown should not fire onDisconnected events") Author: Shixiong Zhu <shixi...@databricks.com> Closes #12481 from zsxwing/SPARK-146

spark git commit: [SPARK-13904] Add exit code parameter to exitExecutor()

2016-04-19 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 9ee95b6ec -> e89633605 [SPARK-13904] Add exit code parameter to exitExecutor() ## What changes were proposed in this pull request? This PR adds exit code parameter to exitExecutor() so that caller can specify different exit code. ## How

spark git commit: [SPARK-14713][TESTS] Fix the flaky test NettyBlockTransferServiceSuite

2016-04-18 Thread zsxwing
634 and 27634 to reduce the possibility of port conflicts. - Make `service1` use `service0.port` to bind to avoid the above race condition. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #12477 from zsxwing/SPARK-14713. Proj

spark git commit: [SPARK-14473][SQL] Define analysis rules to catch operations not supported in streaming

2016-04-18 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 432d1399c -> 775cf17ea [SPARK-14473][SQL] Define analysis rules to catch operations not supported in streaming ## What changes were proposed in this pull request? There are many operations that are currently not supported in the

spark git commit: [SPARK-14579][SQL] Fix a race condition in StreamExecution.processAllAvailable

2016-04-12 Thread zsxwing
Zhu <shixi...@databricks.com> Closes #12339 from zsxwing/race-condition. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/768b3d62 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/768b3d62 Diff: http://git-wip-us.a

spark git commit: [SPARK-14556][SQL] Code clean-ups for package o.a.s.sql.execution.streaming.state

2016-04-12 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 111a62474 -> 852bbc6c0 [SPARK-14556][SQL] Code clean-ups for package o.a.s.sql.execution.streaming.state ## What changes were proposed in this pull request? - `StateStoreConf.**max**DeltasForSnapshot` was renamed to

spark git commit: [SPARK-14437][CORE] Use the address that NettyBlockTransferService listens to create BlockManagerId

2016-04-08 Thread zsxwing
nus of this approach is NettyBlockTransferService won't bound to `0.0.0.0` which is much safer. ## How was this patch tested? Manually checked the bound address using local-cluster. Author: Shixiong Zhu <shixi...@databricks.com> Closes #12240 from zsxwing/SPARK-14437. Project: http:

spark git commit: [SQL][TESTS] Fix for flaky test in ContinuousQueryManagerSuite

2016-04-07 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 033d80815 -> 3aa7d7639 [SQL][TESTS] Fix for flaky test in ContinuousQueryManagerSuite ## What changes were proposed in this pull request? The timeouts were lower the other timeouts in the test. Other tests were stable over the last

spark git commit: [SPARK-14288][SQL] Memory Sink for streaming

2016-04-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 5e64dab86 -> 59236e5c5 [SPARK-14288][SQL] Memory Sink for streaming This PR exposes the internal testing `MemorySink` though the data source API. This will allow users to easily test streaming applications in the Spark shell or other

spark git commit: [SPARK-13211][STREAMING] StreamingContext throws NoSuchElementException when created from non-existent checkpoint directory

2016-04-05 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 7d29c72f6 -> 8e5c1cbf2 [SPARK-13211][STREAMING] StreamingContext throws NoSuchElementException when created from non-existent checkpoint directory ## What changes were proposed in this pull request? Take 2: avoid None.get

spark git commit: [SPARK-14397][WEBUI] and tags are nested in LogPage

2016-04-05 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master d35690158 -> e4bd50412 [SPARK-14397][WEBUI] and tags are nested in LogPage ## What changes were proposed in this pull request? In `LogPage`, the content to be rendered is defined as follows. ``` val content =

spark git commit: [SPARK-14316][SQL] StateStoreCoordinator should extend ThreadSafeRpcEndpoint

2016-04-01 Thread zsxwing
sed at the same time. StateStoreCoordinator should use ThreadSafeRpcEndpoint. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #12100 from zsxwing/fix-StateStoreCoordinator. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: h

spark git commit: [SPARK-14242][CORE][NETWORK] avoid copy in compositeBuffer for frame decoder

2016-03-31 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master f0afafdc5 -> 96941b12f [SPARK-14242][CORE][NETWORK] avoid copy in compositeBuffer for frame decoder ## What changes were proposed in this pull request? In this patch, we set the initial `maxNumComponents` to `Integer.MAX_VALUE` instead of

spark git commit: [SPARK-14254][CORE] Add logs to help investigate the network performance

2016-03-29 Thread zsxwing
ent on connecting and resolving host. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #12046 from zsxwing/connection-time. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spa

spark git commit: [SPARK-12181] Check Cached unaligned-access capability before using Unsafe

2016-03-29 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 366cac6fb -> e1f684539 [SPARK-12181] Check Cached unaligned-access capability before using Unsafe ## What changes were proposed in this pull request? For MemoryMode.OFF_HEAP, Unsafe.getInt etc. are used with no restriction. However, the

spark git commit: [SPARK-14131][SQL] Add a workaround for HADOOP-10622 to fix DataFrameReaderWriterSuite

2016-03-25 Thread zsxwing
all Shell.runCommand now: - offsetLog.add - FileStreamSource.getOffset They will create a file using HDFS API and call Shell.runCommand to set the file permission. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11940 from zsxwing/w

spark git commit: [SQL][HOTFIX] Fix flakiness in StateStoreRDDSuite

2016-03-25 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master b5f8c36e3 -> 11fa8741c [SQL][HOTFIX] Fix flakiness in StateStoreRDDSuite ## What changes were proposed in this pull request? StateStoreCoordinator.reportActiveInstance is async, so subsequence state checks must be in eventually. ## How

spark git commit: [SPARK-14025][STREAMING][WEBUI] Fix streaming job descriptions on the event timeline

2016-03-23 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 69bc2c17f -> de4e48b62 [SPARK-14025][STREAMING][WEBUI] Fix streaming job descriptions on the event timeline ## What changes were proposed in this pull request? Removed the extra `...` for each streaming job's description on the event

spark git commit: [MINOR] a minor fix for the comments of a method in RPC Dispatcher

2016-03-15 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 50e3644d0 -> dddf2f2d8 [MINOR] a minor fix for the comments of a method in RPC Dispatcher ## What changes were proposed in this pull request? a minor fix for the comments of a method in RPC Dispatcher ## How was this patch tested?

spark git commit: [SPARK-13747][SQL] Fix concurrent query with fork-join pool

2016-03-09 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master dbf2a7cfa -> 37fcda3e6 [SPARK-13747][SQL] Fix concurrent query with fork-join pool ## What changes were proposed in this pull request? Fix this use case, which was already fixed in SPARK-10548 in 1.6 but was broken in master due to

spark git commit: [SPARK-13711][CORE] Don't call SparkUncaughtExceptionHandler in AppClient as it's in driver

2016-03-07 Thread zsxwing
all `Utils.tryOrExit` as it will send exception to SparkUncaughtExceptionHandler and call `System.exit`. This PR just removed `Utils.tryOrExit`. ## How was this patch tested? manual tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11566 from zsxwing/SPARK-13711. Project: http:

spark git commit: [SPARK-13711][CORE] Don't call SparkUncaughtExceptionHandler in AppClient as it's in driver

2016-03-07 Thread zsxwing
xit` as it will send exception to SparkUncaughtExceptionHandler and call `System.exit`. This PR just removed `Utils.tryOrExit`. ## How was this patch tested? manual tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11566 from zsxwing/SPARK-13711. Project: http:

spark git commit: [SPARK-13655] Improve isolation between tests in KinesisBackedBlockRDDSuite

2016-03-07 Thread zsxwing
sts to hang. See #11558 for more details. /cc zsxwing srowen Author: Josh Rosen <joshro...@databricks.com> Closes #11564 from JoshRosen/SPARK-13655. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e9e67b39 Tree: http:

spark git commit: [MINOR][DOC] improve the doc for "spark.memory.offHeap.size"

2016-03-07 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 2434f16cc -> cf4e62ec2 [MINOR][DOC] improve the doc for "spark.memory.offHeap.size" The description of "spark.memory.offHeap.size" in the current document does not clearly state that memory is counted with bytes This PR contains

spark git commit: [MINOR][DOC] improve the doc for "spark.memory.offHeap.size"

2016-03-07 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master e72914f37 -> a3ec50a4b [MINOR][DOC] improve the doc for "spark.memory.offHeap.size" The description of "spark.memory.offHeap.size" in the current document does not clearly state that memory is counted with bytes This PR contains a

spark git commit: [SPARK-13693][STREAMING][TESTS] Stop StreamingContext before deleting checkpoint dir

2016-03-05 Thread zsxwing
How was this patch tested? unit tests Author: Shixiong Zhu <shixi...@databricks.com> Closes #11531 from zsxwing/SPARK-13693. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8290004d Tree: http://git-wip-us.apache.org/repos/asf/s

spark git commit: [SPARK-12073][STREAMING] backpressure rate controller consumes events preferentially from lagg…

2016-03-04 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master a6e2bd31f -> f19228eed [SPARK-12073][STREAMING] backpressure rate controller consumes events preferentially from lagg… …ing partitions I'm pretty sure this is the reason we couldn't easily recover from an unbalanced Kafka partition

spark git commit: [SPARK-13652][CORE] Copy ByteBuffer in sendRpcSync as it will be recycled

2016-03-03 Thread zsxwing
fer will be recycled and reused. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11499 from zsxwing/SPARK-13652. (cherry picked from commit 465c665db1dc65e3b02c584cf7f8d06b24909b0c) Signed-off-by: Shixiong Zhu <shixi...@databricks.co

spark git commit: [SPARK-13652][CORE] Copy ByteBuffer in sendRpcSync as it will be recycled

2016-03-03 Thread zsxwing
fer will be recycled and reused. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11499 from zsxwing/SPARK-13652. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/465c665d T

spark git commit: [SPARK-13584][SQL][TESTS] Make ContinuousQueryManagerSuite not output logs to the console

2016-03-03 Thread zsxwing
; Closes #11439 from zsxwing/quietly-ContinuousQueryManagerSuite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ad0de99f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ad0de99f Diff: http://git-wip-us.apache.org/repos/

spark git commit: [SPARK-13468][WEB UI] Fix a corner case where the Stage UI page should show DAG but it doesn't show

2016-02-25 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 35316cb0b -> dc6c5ea4c [SPARK-13468][WEB UI] Fix a corner case where the Stage UI page should show DAG but it doesn't show When uses clicks more than one time on any stage in the DAG graph on the *Job* web UI page, many new *Stage* web

spark git commit: [SPARK-13069][STREAMING] Add "ask" style store() to ActorReciever

2016-02-25 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 751724b13 -> fb8bb0476 [SPARK-13069][STREAMING] Add "ask" style store() to ActorReciever Introduces a "ask" style ```store``` in ```ActorReceiver``` as a way to allow actor receiver blocked by back pressure or maxRate. Author: Lin Zhao

spark git commit: [SPARK-13464][STREAMING][PYSPARK] Fix failed streaming in pyspark in branch 1.3

2016-02-25 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.3 387d81891 -> 6ddde8eda [SPARK-13464][STREAMING][PYSPARK] Fix failed streaming in pyspark in branch 1.3 JIRA: https://issues.apache.org/jira/browse/SPARK-13464 ## What changes were proposed in this pull request? During backport a

spark git commit: Revert "[SPARK-13117][WEB UI] WebUI should use the local ip not 0.0.0.0"

2016-02-25 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 5fcf4c2bf -> 46f6e7931 Revert "[SPARK-13117][WEB UI] WebUI should use the local ip not 0.0.0.0" This reverts commit 2e44031fafdb8cf486573b98e4faa6b31ffb90a4. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit:

spark git commit: [SPARK-11627] Add initial input rate limit for spark streaming backpressure mechanism.

2016-02-16 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 5f37aad48 -> 7218c0eba [SPARK-11627] Add initial input rate limit for spark streaming backpressure mechanism. https://issues.apache.org/jira/browse/SPARK-11627 Spark Streaming backpressure mechanism has no initial input rate limit, it

spark git commit: [SPARK-13308] ManagedBuffers passed to OneToOneStreamManager need to be freed in non-error cases

2016-02-16 Thread zsxwing
ant network code so that the ManagedBuffers are freed as soon as the messages containing them are processed by the lower-level Netty message sending code. /cc zsxwing for review. Author: Josh Rosen <joshro...@databricks.com> Closes #11193 from JoshRosen/add-missing-release-calls-in-netw

spark git commit: [SPARK-6166] Limit number of in flight outbound requests

2016-02-11 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master a2c7dcf61 -> 894921d81 [SPARK-6166] Limit number of in flight outbound requests This JIRA is related to https://github.com/apache/spark/pull/5852 Had to do some minor rework and test to make sure it works with current version of spark.

spark git commit: [STREAMING][TEST] Fix flaky streaming.FailureSuite

2016-02-11 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 13c17cbb0 -> 219a74a7c [STREAMING][TEST] Fix flaky streaming.FailureSuite Under some corner cases, the test suite failed to shutdown the SparkContext causing cascaded failures. This fix does two things - Makes sure no SparkContext is

[2/2] spark git commit: [SPARK-13146][SQL] Management API for continuous queries

2016-02-10 Thread zsxwing
[SPARK-13146][SQL] Management API for continuous queries ### Management API for Continuous Queries **API for getting status of each query** - Whether active or not - Unique name of each query - Status of the sources and sinks - Exceptions **API for managing each query** - Immediately stop an

[1/2] spark git commit: [SPARK-13146][SQL] Management API for continuous queries

2016-02-10 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 29c547303 -> 0902e2028 http://git-wip-us.apache.org/repos/asf/spark/blob/0902e202/sql/core/src/test/scala/org/apache/spark/sql/util/ContinuousQueryListenerSuite.scala --

spark git commit: [SPARK-13245][CORE] Call shuffleMetrics methods only in one thread for ShuffleBlockFetcherIterator

2016-02-09 Thread zsxwing
so as to always use shuffleMetrics in one thread. Also fix a race condition that could cause memory leak. Author: Shixiong Zhu <shixi...@databricks.com> Closes #11138 from zsxwing/SPARK-13245. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/

spark git commit: [SPARK-13166][SQL] Rename DataStreamReaderWriterSuite to DataFrameReaderWriterSuite

2016-02-05 Thread zsxwing
es #11096 from zsxwing/rename. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7b73f171 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7b73f171 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7b73f171 Branch: r

spark git commit: [SPARK-13195][STREAMING] Fix NoSuchElementException when a state is not set but timeoutThreshold is defined

2016-02-04 Thread zsxwing
; Closes #11081 from zsxwing/SPARK-13195. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8e2f2963 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8e2f2963 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8e2f296

spark git commit: [SPARK-3611][WEB UI] Show number of cores for each executor in application web UI

2016-02-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 9dd2741eb -> 3221eddb8 [SPARK-3611][WEB UI] Show number of cores for each executor in application web UI Added a Cores column in the Executors UI Author: Alex Bozarth Closes #11039 from ajbozarth/spark3611.

spark git commit: [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns

2016-02-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 138c300f9 -> e9eb248ed [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns I have clearly prefix the two 'Duration' columns in 'Details of Batch' Streaming tab as 'Output Op Duration' and 'Job Duration'

spark git commit: [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns

2016-02-03 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 2f8abb4af -> 5fe8796c2 [SPARK-12739][STREAMING] Details of batch in Streaming tab uses two Duration columns I have clearly prefix the two 'Duration' columns in 'Details of Batch' Streaming tab as 'Output Op Duration' and 'Job

spark git commit: [SPARK-13121][STREAMING] java mapWithState mishandles scala Option

2016-02-02 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master be5dd881f -> d0df2ca40 [SPARK-13121][STREAMING] java mapWithState mishandles scala Option Already merged into 1.6 branch, this PR is to commit to master the same change Author: Gabriele Nizzoli Closes #11028 from

spark git commit: [SPARK-13121][STREAMING] java mapWithState mishandles scala Option

2016-02-02 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 53f518a6e -> 4c28b4c8f [SPARK-13121][STREAMING] java mapWithState mishandles scala Option java mapwithstate with Function3 has wrong conversion of java `Optional` to scala `Option`, fixed code uses same conversion used in the

spark git commit: [SPARK-13082][PYSPARK] Backport the fix of 'read.json(rdd)' in #10559 to branch-1.6

2016-01-29 Thread zsxwing
fix of 'read.json(rdd)' to branch-1.6. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10988 from zsxwing/json-rdd. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/84dab726 Tree: http://git-wip-us.apache.org/repos/

spark git commit: [SPARK-13055] SQLHistoryListener throws ClassCastException

2016-01-29 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 2b027e9a3 -> e38b0baa3 [SPARK-13055] SQLHistoryListener throws ClassCastException This is an existing issue uncovered recently by #10835. The reason for the exception was because the `SQLHistoryListener` gets all sorts of accumulators,

spark git commit: [SPARK-12967][NETTY] Avoid NettyRpc error message during sparkContext shutdown

2016-01-26 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 58f5d8c1d -> bae3c9a4e [SPARK-12967][NETTY] Avoid NettyRpc error message during sparkContext shutdown If there's an RPC issue while sparkContext is alive but stopped (which would happen only when executing SparkContext.stop), log a

spark git commit: [SPARK-12614][CORE] Don't throw non fatal exception from ask

2016-01-26 Thread zsxwing
ion from RpcEndpointRef.ask. We can send the exception to the future for `ask`. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10568 from zsxwing/send-ask-fail. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit

spark git commit: [HOTFIX]Remove rpcEnv.awaitTermination to avoid dead-lock in some test

2016-01-22 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master bc1babd63 -> ea5c38fe7 [HOTFIX]Remove rpcEnv.awaitTermination to avoid dead-lock in some test Looks rpcEnv.awaitTermination may block some tests forever. Just remove it and investigate the tests. Project:

spark git commit: Revert "[SPARK-12829] Turn Java style checker on"

2016-01-18 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master a973f483f -> 4bcea1b85 Revert "[SPARK-12829] Turn Java style checker on" This reverts commit 591c88c9e2a6c2e2ca84f1b66c635f198a16d112. `lint-java` doesn't work on a machine with a clean Maven cache. Project:

spark git commit: [SPARK-12784][UI] Fix Spark UI IndexOutOfBoundsException with dynamic allocation

2016-01-14 Thread zsxwing
ks.com> Closes #10728 from zsxwing/SPARK-12784. (cherry picked from commit 501e99ef0fbd2f2165095548fe67a3447ccbfc91) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commi

spark git commit: [SPARK-12784][UI] Fix Spark UI IndexOutOfBoundsException with dynamic allocation

2016-01-14 Thread zsxwing
ks.com> Closes #10728 from zsxwing/SPARK-12784. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/501e99ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/501e99ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff

spark git commit: [SPARK-12784][UI] Fix Spark UI IndexOutOfBoundsException with dynamic allocation

2016-01-14 Thread zsxwing
ks.com> Closes #10728 from zsxwing/SPARK-12784. (cherry picked from commit 501e99ef0fbd2f2165095548fe67a3447ccbfc91) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commi

spark git commit: [STREAMING][MINOR] Typo fixes

2016-01-11 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 d4cfd2acd -> ce906b33d [STREAMING][MINOR] Typo fixes Author: Jacek Laskowski Closes #10698 from jaceklaskowski/streaming-kafka-typo-fixes. (cherry picked from commit b313badaa049f847f33663c61cd70ee2f2cbebac)

spark git commit: [STREAMING][MINOR] Typo fixes

2016-01-11 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 9559ac5f7 -> b313badaa [STREAMING][MINOR] Typo fixes Author: Jacek Laskowski Closes #10698 from jaceklaskowski/streaming-kafka-typo-fixes. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit:

spark git commit: [SPARK-12701][CORE] FileAppender should use join to ensure writing thread completion

2016-01-08 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master cfe1ba56e -> ea104b8f1 [SPARK-12701][CORE] FileAppender should use join to ensure writing thread completion Changed Logging FileAppender to use join in `awaitTermination` to ensure that thread is properly finished before returning.

spark git commit: [SPARK-12510][STREAMING] Refactor ActorReceiver to support Java

2016-01-07 Thread zsxwing
ver` for Scala and `JavaActorReceiver` for Java 4. Add `JavaActorWordCount` example Author: Shixiong Zhu <shixi...@databricks.com> Closes #10457 from zsxwing/java-actor-stream. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit

spark git commit: [SPARK-12673][UI] Add missing uri prepending for job description

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 11b901b22 -> 94af69c9b [SPARK-12673][UI] Add missing uri prepending for job description Otherwise the url will be failed to proxy to the right one if in YARN mode. Here is the screenshot: ![screen shot 2016-01-06 at 5 28 26

spark git commit: [SPARK-12673][UI] Add missing uri prepending for job description

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.5 5e86c0cce -> f2bc02ec4 [SPARK-12673][UI] Add missing uri prepending for job description Otherwise the url will be failed to proxy to the right one if in YARN mode. Here is the screenshot: ![screen shot 2016-01-06 at 5 28 26

spark git commit: [SPARK-12673][UI] Add missing uri prepending for job description

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 8e19c7663 -> 174e72cec [SPARK-12673][UI] Add missing uri prepending for job description Otherwise the url will be failed to proxy to the right one if in YARN mode. Here is the screenshot: ![screen shot 2016-01-06 at 5 28 26

spark git commit: [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming

2016-01-06 Thread zsxwing
;shixi...@databricks.com> Closes #10621 from zsxwing/SPARK-12617-2. (cherry picked from commit 1e6648d62fb82b708ea54c51cd23bfe4f542856e) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/

spark git commit: [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming

2016-01-06 Thread zsxwing
;shixi...@databricks.com> Closes #10621 from zsxwing/SPARK-12617-2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1e6648d6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1e6648d6 Diff: http://git-wip-us.apache.org/

spark git commit: [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming

2016-01-06 Thread zsxwing
;shixi...@databricks.com> Closes #10621 from zsxwing/SPARK-12617-2. (cherry picked from commit 1e6648d62fb82b708ea54c51cd23bfe4f542856e) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/

spark git commit: Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url."

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 8f0ead3e7 -> 39b0a3480 Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url." This reverts commit 8f0ead3e79beb2c5f2731ceaa34fe1c133763386. Will merge #10618

spark git commit: Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url."

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 19e4e9feb -> cbaea9591 Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url." This reverts commit 19e4e9febf9bb4fd69f6d7bc13a54844e4e096f1. Will merge #10618 instead.

spark git commit: Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url."

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.5 fb421af08 -> d10b9d572 Revert "[SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url." This reverts commit fb421af08de73e4ae6b04a576721109cae561865. Will merge #10618

spark git commit: [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url.

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 d821fae0e -> 8f0ead3e7 [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url. Author: huangzhaowei Closes #10617 from SaintBacchus/SPARK-12672.

spark git commit: [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url.

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.5 598a5c2cc -> fb421af08 [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url. Author: huangzhaowei Closes #10617 from SaintBacchus/SPARK-12672.

spark git commit: [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url.

2016-01-06 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 1e6648d62 -> 19e4e9feb [SPARK-12672][STREAMING][UI] Use the uiRoot function instead of default root path to gain the streaming batch url. Author: huangzhaowei Closes #10617 from SaintBacchus/SPARK-12672.

spark git commit: [SPARK-12608][STREAMING] Remove submitJobThreadPool since submitJob doesn't create a separate thread to wait for the job result

2016-01-04 Thread zsxwing
ult. `submitJobThreadPool` was a workaround in `ReceiverTracker` to run these waiting-job-result threads. Now #9264 has been merged to master and resolved this blocking issue, `submitJobThreadPool` can be removed now. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10560 from zsxwi

spark git commit: [SPARK-12490][CORE] Limit the css style scope to fix the Streaming UI

2015-12-29 Thread zsxwing
f8b-39df08426bf8.png;> This PR just added a class for the new style and only applied them to the paged tables. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10517 from zsxwing/fix-streaming-ui. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-u

spark git commit: [SPARK-12489][CORE][SQL][MLIB] Fix minor issues found by FindBugs

2015-12-28 Thread zsxwing
zed` and `ReentrantLock`. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10440 from zsxwing/findbugs. (cherry picked from commit 710b41172958a0b3a2b70c48821aefc81893731b) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit:

spark git commit: [SPARK-12489][CORE][SQL][MLIB] Fix minor issues found by FindBugs

2015-12-28 Thread zsxwing
ock`. Author: Shixiong Zhu <shixi...@databricks.com> Closes #10440 from zsxwing/findbugs. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/710b4117 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/710b4117 Diff: h

spark git commit: [SPARK-11985][STREAMING][KINESIS][DOCS] Update Kinesis docs

2015-12-18 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master 6eba65525 -> 2377b707f [SPARK-11985][STREAMING][KINESIS][DOCS] Update Kinesis docs - Provide example on `message handler` - Provide bit on KPL record de-aggregation - Fix typos Author: Burak Yavuz Closes #9970 from

spark git commit: [SPARK-11985][STREAMING][KINESIS][DOCS] Update Kinesis docs

2015-12-18 Thread zsxwing
Repository: spark Updated Branches: refs/heads/branch-1.6 bd33d4ee8 -> eca401ee5 [SPARK-11985][STREAMING][KINESIS][DOCS] Update Kinesis docs - Provide example on `message handler` - Provide bit on KPL record de-aggregation - Fix typos Author: Burak Yavuz Closes #9970

spark git commit: [SPARK-12220][CORE] Make Utils.fetchFile support files that contain special characters

2015-12-17 Thread zsxwing
0208 from zsxwing/uri. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/86e405f3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/86e405f3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/86e405f3 Branch: refs/hea

spark git commit: [SPARK-12220][CORE] Make Utils.fetchFile support files that contain special characters

2015-12-17 Thread zsxwing
es #10208 from zsxwing/uri. (cherry picked from commit 86e405f357711ae93935853a912bc13985c259db) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1fbca412 Tree:

spark git commit: [SPARK-12410][STREAMING] Fix places that use '.' and '|' directly in split

2015-12-17 Thread zsxwing
..@databricks.com> Closes #10361 from zsxwing/reg-bug. (cherry picked from commit 540b5aeadc84d1a5d61bda4414abd6bf35dc7ff9) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/c

spark git commit: [SPARK-12410][STREAMING] Fix places that use '.' and '|' directly in split

2015-12-17 Thread zsxwing
..@databricks.com> Closes #10361 from zsxwing/reg-bug. (cherry picked from commit 540b5aeadc84d1a5d61bda4414abd6bf35dc7ff9) Signed-off-by: Shixiong Zhu <shixi...@databricks.com> Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/c

spark git commit: [SPARK-12376][TESTS] Spark Streaming Java8APISuite fails in assertOrderInvariantEquals method

2015-12-17 Thread zsxwing
Repository: spark Updated Branches: refs/heads/master e096a652b -> ed6ebda5c [SPARK-12376][TESTS] Spark Streaming Java8APISuite fails in assertOrderInvariantEquals method org.apache.spark.streaming.Java8APISuite.java is failing due to trying to sort immutable list in

<    2   3   4   5   6   7   8   >