Commit graph

4837 commits

Author SHA1 Message Date
Tathagata Das bc9a0e0323 [SPARK-9967] [SPARK-10099] [STREAMING] Renamed conf spark.streaming.backpressure.{enable-->enabled} and fixed deprecated annotations
Small changes
- Renamed conf spark.streaming.backpressure.{enable --> enabled}
- Change Java Deprecated annotations to Scala deprecated annotation with more information.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #8299 from tdas/SPARK-9967.
2015-08-18 23:37:57 -07:00
Josh Rosen 010b03ed52 [SPARK-9952] Fix N^2 loop when DAGScheduler.getPreferredLocsInternal accesses cacheLocs
In Scala, `Seq.fill` always seems to return a List. Accessing a list by index is an O(N) operation. Thus, the following code will be really slow (~10 seconds on my machine):

```scala
val numItems = 100000
val s = Seq.fill(numItems)(1)
for (i <- 0 until numItems) s(i)
```

It turns out that we had a loop like this in DAGScheduler code, although it's a little tricky to spot. In `getPreferredLocsInternal`, there's a call to `getCacheLocs(rdd)(partition)`.  The `getCacheLocs` call returns a Seq. If this Seq is a List and the RDD contains many partitions, then indexing into this list will cost O(partitions). Thus, when we loop over our tasks to compute their individual preferred locations we implicitly perform an N^2 loop, reducing scheduling throughput.

This patch fixes this by replacing `Seq` with `Array`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8178 from JoshRosen/dagscheduler-perf.
2015-08-18 22:30:13 -07:00
Marcelo Vanzin c1840a862e [SPARK-7736] [CORE] Fix a race introduced in PythonRunner.
The fix for SPARK-7736 introduced a race where a port value of "-1"
could be passed down to the pyspark process, causing it to fail to
connect back to the JVM. This change adds code to fix that race.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8258 from vanzin/SPARK-7736.
2015-08-18 11:36:36 -07:00
CodingCat c34e9ff0ea [MINOR] fix the comments in IndexShuffleBlockResolver
it might be a typo  introduced at the first moment or some leftover after some renaming......

the name of the method accessing the index file is called `getBlockData` now (not `getBlockLocation` as indicated in the comments)

Author: CodingCat <zhunansjtu@gmail.com>

Closes #8238 from CodingCat/minor_1.
2015-08-18 10:31:11 +01:00
Marcelo Vanzin f68d024096 [SPARK-7736] [CORE] [YARN] Make pyspark fail YARN app on failure.
The YARN backend doesn't like when user code calls `System.exit`,
since it cannot know the exit status and thus cannot set an
appropriate final status for the application.

So, for pyspark, avoid that call and instead throw an exception with
the exit code. SparkSubmit handles that exception and exits with
the given exit code, while YARN uses the exit code as the failure
code for the Spark app.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #7751 from vanzin/SPARK-9416.
2015-08-17 10:34:22 -07:00
Rohit Agarwal ed092a06c2 [SPARK-9924] [WEB UI] Don't schedule checkForLogs while some of them are already running.
Author: Rohit Agarwal <rohita@qubole.com>

Closes #8153 from mindprince/SPARK-9924.
2015-08-17 10:31:57 -07:00
Calvin Jia 3ff81ad2de [SPARK-9199] [CORE] Upgrade Tachyon version from 0.7.0 -> 0.7.1.
Updates the tachyon-client version to the latest release.

The main difference between 0.7.0 and 0.7.1 on the client side is to support running Tachyon on local file system by default.

No new non-Tachyon dependencies are added, and no code changes are required since the client API has not changed.

Author: Calvin Jia <jia.calvin@gmail.com>

Closes #8235 from calvinjia/spark-9199-master.
2015-08-17 08:28:16 +01:00
Matei Zaharia cf016075a0 [SPARK-10008] Ensure shuffle locality doesn't take precedence over narrow deps
The shuffle locality patch made the DAGScheduler aware of shuffle data,
but for RDDs that have both narrow and shuffle dependencies, it can
cause them to place tasks based on the shuffle dependency instead of the
narrow one. This case is common in iterative join-based algorithms like
PageRank and ALS, where one RDD is hash-partitioned and one isn't.

Author: Matei Zaharia <matei@databricks.com>

Closes #8220 from mateiz/shuffle-loc-fix.
2015-08-16 00:34:58 -07:00
Herman van Hovell a85fb6c07f [SPARK-9980] [BUILD] Fix SBT publishLocal error due to invalid characters in doc
Tiny modification to a few comments ```sbt publishLocal``` work again.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #8209 from hvanhovell/SPARK-9980.
2015-08-15 10:46:04 +01:00
Davies Liu 37586e5449 [HOTFIX] fix duplicated braces
Author: Davies Liu <davies@databricks.com>

Closes #8219 from davies/fix_typo.
2015-08-14 20:56:55 -07:00
Reynold Xin e5fd60415f [SPARK-9934] Deprecate NIO ConnectionManager.
Deprecate NIO ConnectionManager in Spark 1.5.0, before removing it in Spark 1.6.0.

Author: Reynold Xin <rxin@databricks.com>

Closes #8162 from rxin/SPARK-9934.
2015-08-14 20:55:32 -07:00
jerryshao 9407baa2a7 [SPARK-9877] [CORE] Fix StandaloneRestServer NPE when submitting application
Detailed exception log can be seen in [SPARK-9877](https://issues.apache.org/jira/browse/SPARK-9877), the problem is when creating `StandaloneRestServer`, `self` (`masterEndpoint`) is null.  So this fix is creating `StandaloneRestServer` when `self` is available.

Author: jerryshao <sshao@hortonworks.com>

Closes #8127 from jerryshao/SPARK-9877.
2015-08-14 13:44:38 -07:00
Andrew Or 6518ef6303 [SPARK-9948] Fix flaky AccumulatorSuite - internal accumulators
In these tests, we use a custom listener and we assert on fields in the stage / task completion events. However, these events are posted in a separate thread so they're not guaranteed to be posted in time. This commit fixes this flakiness through a job end registration callback.

Author: Andrew Or <andrew@databricks.com>

Closes #8176 from andrewor14/fix-accumulator-suite.
2015-08-14 13:42:53 -07:00
Carson Wang 33bae585d4 [SPARK-9809] Task crashes because the internal accumulators are not properly initialized
When a stage failed and another stage was resubmitted with only part of partitions to compute, all the tasks failed with error message: java.util.NoSuchElementException: key not found: peakExecutionMemory.
This is because the internal accumulators are not properly initialized for this stage while other codes assume the internal accumulators always exist.

Author: Carson Wang <carson.wang@intel.com>

Closes #8090 from carsonwang/SPARK-9809.
2015-08-14 13:38:25 -07:00
Neelesh Srinivas Salian 57c2d08800 [SPARK-9923] [CORE] ShuffleMapStage.numAvailableOutputs should be an Int instead of Long
Modified type of ShuffleMapStage.numAvailableOutputs from Long to Int

Author: Neelesh Srinivas Salian <nsalian@cloudera.com>

Closes #8183 from nssalian/SPARK-9923.
2015-08-14 20:03:50 +01:00
Davies Liu bd35385d53 [SPARK-9945] [SQL] pageSize should be calculated from executor.memory
Currently, pageSize of TungstenSort is calculated from driver.memory, it should use executor.memory instead.

Also, in the worst case, the safeFactor could be 4 (because of rounding), increase it to 16.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #8175 from davies/page_size.
2015-08-13 21:12:59 -07:00
Andrew Or 8815ba2f67 [SPARK-9649] Fix MasterSuite, third time's a charm
This particular test did not load the default configurations so
it continued to start the REST server, which causes port bind
exceptions.
2015-08-13 11:31:10 -07:00
Davies Liu a8ab2634c1 [SPARK-9832] [SQL] add a thread-safe lookup for BytesToBytseMap
This patch add a thread-safe lookup for BytesToBytseMap, and use that in broadcasted HashedRelation.

Author: Davies Liu <davies@databricks.com>

Closes #8151 from davies/safeLookup.
2015-08-12 21:26:00 -07:00
Josh Rosen 7b13ed27c1 [SPARK-9870] Disable driver UI and Master REST server in SparkSubmitSuite
I think that we should pass additional configuration flags to disable the driver UI and Master REST server in SparkSubmitSuite and HiveSparkSubmitSuite. This might cut down on port-contention-related flakiness in Jenkins.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8124 from JoshRosen/disable-ui-in-sparksubmitsuite.
2015-08-12 18:52:11 -07:00
Rohit Agarwal 0d1d146c22 [SPARK-9724] [WEB UI] Avoid unnecessary redirects in the Spark Web UI.
Author: Rohit Agarwal <rohita@qubole.com>

Closes #8014 from mindprince/SPARK-9724 and squashes the following commits:

a7af5ff [Rohit Agarwal] [SPARK-9724] [WEB UI] Inline attachPrefix and attachPrefixForRedirect. Fix logic of attachPrefix
8a977cd [Rohit Agarwal] [SPARK-9724] [WEB UI] Address review comments: Remove unneeded code, update scaladoc.
b257844 [Rohit Agarwal] [SPARK-9724] [WEB UI] Avoid unnecessary redirects in the Spark Web UI.
2015-08-12 17:48:43 -07:00
Michel Lemay ab7e721cfe [SPARK-9826] [CORE] Fix cannot use custom classes in log4j.properties
Refactor Utils class and create ShutdownHookManager.

NOTE: Wasn't able to run /dev/run-tests on windows machine.
Manual tests were conducted locally using custom log4j.properties file with Redis appender and logstash formatter (bundled in the fat-jar submitted to spark)

ex:
log4j.rootCategory=WARN,console,redis
log4j.appender.console=org.apache.log4j.ConsoleAppender
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n

log4j.logger.org.eclipse.jetty=WARN
log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
log4j.logger.org.apache.spark.graphx.Pregel=INFO

log4j.appender.redis=com.ryantenney.log4j.FailoverRedisAppender
log4j.appender.redis.endpoints=hostname:port
log4j.appender.redis.key=mykey
log4j.appender.redis.alwaysBatch=false
log4j.appender.redis.layout=net.logstash.log4j.JSONEventLayoutV1

Author: michellemay <mlemay@gmail.com>

Closes #8109 from michellemay/SPARK-9826.
2015-08-12 16:41:35 -07:00
Niranjan Padmanabhan 738f353988 [SPARK-9092] Fixed incompatibility when both num-executors and dynamic...
… allocation are set. Now, dynamic allocation is set to false when num-executors is explicitly specified as an argument. Consequently, executorAllocationManager in not initialized in the SparkContext.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@cloudera.com>

Closes #7657 from neurons/SPARK-9092.
2015-08-12 16:10:21 -07:00
Xiangrui Meng 6f60298b1d [SPARK-8967] [DOC] add Since annotation
Add `Since` as a Scala annotation. The benefit is that we can use it without having explicit JavaDoc. This is useful for inherited methods. The limitation is that is doesn't show up in the generated Java API documentation. This might be fixed by modifying genjavadoc. I think we could leave it as a TODO.

This is how the generated Scala doc looks:

`since` JavaDoc tag:

![screen shot 2015-08-11 at 10 00 37 pm](https://cloud.githubusercontent.com/assets/829644/9230761/fa72865c-40d8-11e5-807e-0f3c815c5acd.png)

`Since` annotation:

![screen shot 2015-08-11 at 10 00 28 pm](https://cloud.githubusercontent.com/assets/829644/9230764/0041d7f4-40d9-11e5-8124-c3f3e5d5b31f.png)

rxin

Author: Xiangrui Meng <meng@databricks.com>

Closes #8131 from mengxr/SPARK-8967.
2015-08-12 14:28:23 -07:00
Andrew Or e0110792ef [SPARK-9747] [SQL] Avoid starving an unsafe operator in aggregation
This is the sister patch to #8011, but for aggregation.

In a nutshell: create the `TungstenAggregationIterator` before computing the parent partition. Internally this creates a `BytesToBytesMap` which acquires a page in the constructor as of this patch. This ensures that the aggregation operator is not starved since we reserve at least 1 page in advance.

rxin yhuai

Author: Andrew Or <andrew@databricks.com>

Closes #8038 from andrewor14/unsafe-starve-memory-agg.
2015-08-12 10:08:35 -07:00
Andrew Or be5d191207 [SPARK-9795] Dynamic allocation: avoid double counting when killing same executor twice
This is based on KaiXinXiaoLei's changes in #7716.

The issue is that when someone calls `sc.killExecutor("1")` on the same executor twice quickly, then the executor target will be adjusted downwards by 2 instead of 1 even though we're only actually killing one executor. In certain cases where we don't adjust the target back upwards quickly, we'll end up with jobs hanging.

This is a common danger because there are many places where this is called:
- `HeartbeatReceiver` kills an executor that has not been sending heartbeats
- `ExecutorAllocationManager` kills an executor that has been idle
- The user code might call this, which may interfere with the previous callers

While it's not clear whether this fixes SPARK-9745, fixing this potential race condition seems like a strict improvement. I've added a regression test to illustrate the issue.

Author: Andrew Or <andrew@databricks.com>

Closes #8078 from andrewor14/da-double-kill.
2015-08-12 09:24:50 -07:00
Tom White 2e680668f7 [SPARK-8625] [CORE] Propagate user exceptions in tasks back to driver
This allows clients to retrieve the original exception from the
cause field of the SparkException that is thrown by the driver.
If the original exception is not in fact Serializable then it will
not be returned, but the message and stacktrace will be. (All Java
Throwables implement the Serializable interface, but this is no
guarantee that a particular implementation can actually be
serialized.)

Author: Tom White <tom@cloudera.com>

Closes #7014 from tomwhite/propagate-user-exceptions.
2015-08-12 10:07:11 -05:00
Timothy Chen 5c99d8bf98 [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesos
Some users like to download additional files in their sandbox that they can refer to from their spark program, or even later mount these files to another directory.

Author: Timothy Chen <tnachen@gmail.com>

Closes #7195 from tnachen/mesos_files.
2015-08-11 23:26:33 -07:00
Carson Wang bab8923285 [SPARK-9426] [WEBUI] Job page DAG visualization is not shown
To reproduce the issue, go to the stage page and click DAG Visualization once, then go to the job page to show the job DAG visualization. You will only see the first stage of the job.
Root cause: the java script use local storage to remember your selection. Once you click the stage DAG visualization, the local storage set `expand-dag-viz-arrow-stage` to true. When you go to the job page, the js checks `expand-dag-viz-arrow-stage` in the local storage first and will try to show stage DAG visualization on the job page.
To fix this, I set an id to the DAG span to differ job page and stage page. In the js code, we check the id and local storage together to make sure we show the correct DAG visualization.

Author: Carson Wang <carson.wang@intel.com>

Closes #8104 from carsonwang/SPARK-9426.
2015-08-11 23:25:02 -07:00
zsxwing 4e3f4b934f [SPARK-9829] [WEBUI] Display the update value for peak execution memory
The peak execution memory is not correct because it shows the sum of finished tasks' values when a task finishes.

This PR fixes it by using the update value rather than the accumulator value.

Author: zsxwing <zsxwing@gmail.com>

Closes #8121 from zsxwing/SPARK-9829.
2015-08-11 23:23:17 -07:00
Rohit Agarwal a807fcbe50 [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple applications
Author: Rohit Agarwal <rohita@qubole.com>

Closes #8088 from mindprince/SPARK-9806.
2015-08-11 23:20:39 -07:00
xutingjun b85f9a242a [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasks
Author: xutingjun <xutingjun@huawei.com>
Author: meiyoula <1039320815@qq.com>

Closes #6817 from XuTingjun/SPARK-8366.
2015-08-11 23:19:35 -07:00
zsxwing f16bc68dfb [SPARK-9824] [CORE] Fix the issue that InternalAccumulator leaks WeakReference
`InternalAccumulator.create` doesn't call `registerAccumulatorForCleanup` to register itself with ContextCleaner, so `WeakReference`s for these accumulators in `Accumulators.originals` won't be removed.

This PR added `registerAccumulatorForCleanup` for internal accumulators to avoid the memory leak.

Author: zsxwing <zsxwing@gmail.com>

Closes #8108 from zsxwing/internal-accumulators-leak.
2015-08-11 14:06:23 -07:00
Jeff Zhang bce72797f3 Fix comment error
API is updated but its doc comment is not updated.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #8097 from zjffdu/dev.
2015-08-11 10:42:17 -07:00
Reynold Xin d378396f86 [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
PlatformDependent.UNSAFE is way too verbose.

Author: Reynold Xin <rxin@databricks.com>

Closes #8094 from rxin/SPARK-9815 and squashes the following commits:

229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
2015-08-11 08:41:06 -07:00
Marcelo Vanzin 0f3366a4c7 [SPARK-9710] [TEST] Fix RPackageUtilsSuite when R is not available.
RUtils.isRInstalled throws an exception if R is not installed,
instead of returning false. Fix that.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8008 from vanzin/SPARK-9710 and squashes the following commits:

df72d8c [Marcelo Vanzin] [SPARK-9710] [test] Fix RPackageUtilsSuite when R is not available.
2015-08-10 10:10:40 -07:00
Shivaram Venkataraman 46025616b4 [CORE] [SPARK-9760] Use Option instead of Some for Ivy repos
This was introduced in #7599

cc rxin brkyvz

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #8055 from shivaram/spark-packages-repo-fix and squashes the following commits:

890f306 [Shivaram Venkataraman] Remove test case
51d69ee [Shivaram Venkataraman] Add test case for --packages without --repository
c02e0b4 [Shivaram Venkataraman] Use Option instead of Some for Ivy repos
2015-08-09 14:30:30 -07:00
Reynold Xin e9c36938ba [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
In order for this to work, I had to disable gap sampling.

Author: Reynold Xin <rxin@databricks.com>

Closes #8040 from rxin/SPARK-9752 and squashes the following commits:

f9e248c [Reynold Xin] Fix the test case for real this time.
adbccb3 [Reynold Xin] Fixed test case.
589fb23 [Reynold Xin] Merge branch 'SPARK-9752' of github.com:rxin/spark into SPARK-9752
55ccddc [Reynold Xin] Fixed core test.
78fa895 [Reynold Xin] [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
c9e7112 [Reynold Xin] [SPARK-9752][SQL] Support UnsafeRow in Sample operator.
2015-08-09 10:58:36 -07:00
Carson Wang ef062c1599 [SPARK-9731] Standalone scheduling incorrect cores if spark.executor.cores is not set
The issue only happens if `spark.executor.cores` is not set and executor memory is set to a high value.
For example, if we have a worker with 4G and 10 cores and we set `spark.executor.memory` to 3G, then only 1 core is assigned to the executor. The correct number should be 10 cores.
I've added a unit test to illustrate the issue.

Author: Carson Wang <carson.wang@intel.com>

Closes #8017 from carsonwang/SPARK-9731 and squashes the following commits:

d09ec48 [Carson Wang] Fix code style
86b651f [Carson Wang] Simplify the code
943cc4c [Carson Wang] fix scheduling correct cores to executors
2015-08-07 23:36:26 -07:00
Andrew Or 881548ab20 [SPARK-9674] Re-enable ignored test in SQLQuerySuite
The original code that this test tests is removed in 9270bd06fd. It was ignored shortly before that so we never caught it. This patch re-enables the test and adds the code necessary to make it pass.

JoshRosen yhuai

Author: Andrew Or <andrew@databricks.com>

Closes #8015 from andrewor14/SPARK-9674 and squashes the following commits:

225eac2 [Andrew Or] Merge branch 'master' of github.com:apache/spark into SPARK-9674
8c24209 [Andrew Or] Fix NPE
e541d64 [Andrew Or] Track aggregation memory for both sort and hash
0be3a42 [Andrew Or] Fix test
2015-08-07 14:20:13 -07:00
zsxwing ebfd91c542 [SPARK-9467][SQL]Add SQLMetric to specialize accumulators to avoid boxing
This PR adds SQLMetric/SQLMetricParam/SQLMetricValue to specialize accumulators to avoid boxing. All SQL metrics should use these classes rather than `Accumulator`.

Author: zsxwing <zsxwing@gmail.com>

Closes #7996 from zsxwing/sql-accu and squashes the following commits:

14a5f0a [zsxwing] Address comments
367ca23 [zsxwing] Use localValue directly to avoid changing Accumulable
42f50c3 [zsxwing] Add SQLMetric to specialize accumulators to avoid boxing
2015-08-07 00:09:58 -07:00
Davies Liu 15bd6f338d [SPARK-9453] [SQL] support records larger than page size in UnsafeShuffleExternalSorter
This patch follows exactly #7891 (except testing)

Author: Davies Liu <davies@databricks.com>

Closes #8005 from davies/larger_record and squashes the following commits:

f9c4aff [Davies Liu] address comments
9de5c72 [Davies Liu] support records larger than page size in UnsafeShuffleExternalSorter
2015-08-06 23:40:38 -07:00
Reynold Xin 4309262ec9 [SPARK-9700] Pick default page size more intelligently.
Previously, we use 64MB as the default page size, which was way too big for a lot of Spark applications (especially for single node).

This patch changes it so that the default page size, if unset by the user, is determined by the number of cores available and the total execution memory available.

Author: Reynold Xin <rxin@databricks.com>

Closes #8012 from rxin/pagesize and squashes the following commits:

16f4756 [Reynold Xin] Fixed failing test.
5afd570 [Reynold Xin] private...
0d5fb98 [Reynold Xin] Update default value.
674a6cd [Reynold Xin] Address review feedback.
dc00e05 [Reynold Xin] Merge with master.
73ebdb6 [Reynold Xin] [SPARK-9700] Pick default page size more intelligently.
2015-08-06 23:18:29 -07:00
zsxwing 672f467668 [SPARK-8057][Core]Call TaskAttemptContext.getTaskAttemptID using Reflection
Someone may use the Spark core jar in the maven repo with hadoop 1. SPARK-2075 has already resolved the compatibility issue to support it. But `SparkHadoopMapRedUtil.commitTask` broke it recently.

This PR uses Reflection to call `TaskAttemptContext.getTaskAttemptID` to fix the compatibility issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #6599 from zsxwing/SPARK-8057 and squashes the following commits:

f7a343c [zsxwing] Remove the redundant import
6b7f1af [zsxwing] Call TaskAttemptContext.getTaskAttemptID using Reflection
2015-08-06 21:42:42 -07:00
Andrew Or 014a9f9d8c [SPARK-9709] [SQL] Avoid starving unsafe operators that use sort
The issue is that a task may run multiple sorts, and the sorts run by the child operator (i.e. parent RDD) may acquire all available memory such that other sorts in the same task do not have enough to proceed. This manifests itself in an `IOException("Unable to acquire X bytes of memory")` thrown by `UnsafeExternalSorter`.

The solution is to reserve a page in each sorter in the chain before computing the child operator's (parent RDD's) partitions. This requires us to use a new special RDD that does some preparation before computing the parent's partitions.

Author: Andrew Or <andrew@databricks.com>

Closes #8011 from andrewor14/unsafe-starve-memory and squashes the following commits:

35b69a4 [Andrew Or] Simplify test
0b07782 [Andrew Or] Minor: update comments
5d5afdf [Andrew Or] Merge branch 'master' of github.com:apache/spark into unsafe-starve-memory
254032e [Andrew Or] Add tests
234acbd [Andrew Or] Reserve a page in sorter when preparing each partition
b889e08 [Andrew Or] MapPartitionsWithPreparationRDD
2015-08-06 19:04:57 -07:00
Reynold Xin b87825310a [SPARK-9692] Remove SqlNewHadoopRDD's generated Tuple2 and InterruptibleIterator.
A small performance optimization – we don't need to generate a Tuple2 and then immediately discard the key. We also don't need an extra wrapper from InterruptibleIterator.

Author: Reynold Xin <rxin@databricks.com>

Closes #8000 from rxin/SPARK-9692 and squashes the following commits:

1d4d0b3 [Reynold Xin] [SPARK-9692] Remove SqlNewHadoopRDD's generated Tuple2 and InterruptibleIterator.
2015-08-06 18:25:38 -07:00
Marcelo Vanzin e234ea1b49 [SPARK-9645] [YARN] [CORE] Allow shuffle service to read shuffle files.
Spark should not mess with the permissions of directories created
by the cluster manager. Here, by setting the block manager dir
permissions to 700, the shuffle service (running as the YARN user)
wouldn't be able to serve shuffle files created by applications.

Also, the code to protect the local app dir was missing in standalone's
Worker; that has been now added. Since all processes run as the same
user in standalone, `chmod 700` should not cause problems.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #7966 from vanzin/SPARK-9645 and squashes the following commits:

6e07b31 [Marcelo Vanzin] Protect the app dir in standalone mode.
384ba6a [Marcelo Vanzin] [SPARK-9645] [yarn] [core] Allow shuffle service to read shuffle files.
2015-08-06 15:30:27 -07:00
Tathagata Das 0a078303d0 [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAMING] Make BlockGenerator more robust and make all BlockGenerators subscribe to rate limit updates
In some receivers, instead of using the default `BlockGenerator` in `ReceiverSupervisorImpl`, custom generator with their custom listeners are used for reliability (see [`ReliableKafkaReceiver`](https://github.com/apache/spark/blob/master/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala#L99) and [updated `KinesisReceiver`](https://github.com/apache/spark/pull/7825/files)). These custom generators do not receive rate updates. This PR modifies the code to allow custom `BlockGenerator`s to be created through the `ReceiverSupervisorImpl` so that they can be kept track and rate updates can be applied.

In the process, I did some simplification, and de-flaki-fication of some rate controller related tests. In particular.
- Renamed `Receiver.executor` to `Receiver.supervisor` (to match `ReceiverSupervisor`)
- Made `RateControllerSuite` faster (by increasing batch interval) and less flaky
- Changed a few internal API to return the current rate of block generators as Long instead of Option\[Long\] (was inconsistent at places).
- Updated existing `ReceiverTrackerSuite` to test that custom block generators get rate updates as well.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #7913 from tdas/SPARK-9556 and squashes the following commits:

41d4461 [Tathagata Das] fix scala style
eb9fd59 [Tathagata Das] Updated kinesis receiver
d24994d [Tathagata Das] Updated BlockGeneratorSuite to use manual clock in BlockGenerator
d70608b [Tathagata Das] Updated BlockGenerator with states and proper synchronization
f6bd47e [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9556
31da173 [Tathagata Das] Fix bug
12116df [Tathagata Das] Add BlockGeneratorSuite
74bd069 [Tathagata Das] Fix style
989bb5c [Tathagata Das] Made BlockGenerator fail is used after stop, and added better unit tests for it
3ff618c [Tathagata Das] Fix test
b40eff8 [Tathagata Das] slight refactoring
f0df0f1 [Tathagata Das] Scala style fixes
51759cb [Tathagata Das] Refactored rate controller tests and added the ability to update rate of any custom block generator
2015-08-06 14:35:30 -07:00
Liang-Chi Hsieh 21fdfd7d6f [SPARK-9548][SQL] Add a destructive iterator for BytesToBytesMap
This pull request adds a destructive iterator to BytesToBytesMap. When used, the iterator frees pages as it traverses them. This is part of the effort to avoid starving when we have more than one operators that can exhaust memory.

This is based on #7924, but fixes a bug there (Don't use destructive iterator in UnsafeKVExternalSorter).

Closes #7924.

Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #8003 from rxin/map-destructive-iterator and squashes the following commits:

6b618c3 [Reynold Xin] Don't use destructive iterator in UnsafeKVExternalSorter.
a7bd8ec [Reynold Xin] Merge remote-tracking branch 'viirya/destructive_iter' into map-destructive-iterator
7652083 [Liang-Chi Hsieh] For comments: add destructiveIterator(), modify unit test, remove code block.
4a3e9de [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
581e9e3 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into destructive_iter
f0ff783 [Liang-Chi Hsieh] No need to free last page.
9e9d2a3 [Liang-Chi Hsieh] Add a destructive iterator for BytesToBytesMap.
2015-08-06 14:33:29 -07:00
Yin Huai 4581badbc8 [SPARK-9611] [SQL] Fixes a few corner cases when we spill a UnsafeFixedWidthAggregationMap
This PR has the following three small fixes.

1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty.
2. We will not not spill a InMemorySorter if it is empty.
3. We will not add a SpillReader to a SpillMerger if this SpillReader is empty.

JIRA: https://issues.apache.org/jira/browse/SPARK-9611

Author: Yin Huai <yhuai@databricks.com>

Closes #7948 from yhuai/unsafeEmptyMap and squashes the following commits:

9727abe [Yin Huai] Address Josh's comments.
34b6f76 [Yin Huai] 1. UnsafeKVExternalSorter does not use 0 as the initialSize to create an UnsafeInMemorySorter if its BytesToBytesMap is empty. 2. Do not spill a InMemorySorter if it is empty. 3. Do not add spill to SpillMerger if this spill is empty.
2015-08-05 19:19:09 -07:00
Marcelo Vanzin 4399b7b090 [SPARK-9651] Fix UnsafeExternalSorterSuite.
First, it's probably a bad idea to call generated Scala methods
from Java. In this case, the method being called wasn't actually
"Utils.createTempDir()", but actually the method that returns the
first default argument to the actual createTempDir method, which
is just the location of java.io.tmpdir; meaning that all tests in
the class were using the same temp dir, and thus affecting each
other.

Second, spillingOccursInResponseToMemoryPressure was not writing
enough records to actually cause a spill.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #7970 from vanzin/SPARK-9651 and squashes the following commits:

74d357f [Marcelo Vanzin] Clean up temp dir on test tear down.
a64f36a [Marcelo Vanzin] [SPARK-9651] Fix UnsafeExternalSorterSuite.
2015-08-05 17:58:36 -07:00