| Commit message (Collapse) | Author | Age | Files | Lines |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This patch adds support for giving accumulators user-visible names and displaying accumulator values in the web UI. This allows users to create custom counters that can display in the UI. The current approach displays both the accumulator deltas caused by each task and a "current" value of the accumulator totals for each stage, which gets update as tasks finish.
Currently in Spark developers have been extending the `TaskMetrics` functionality to provide custom instrumentation for RDD's. This provides a potentially nicer alternative of going through the existing accumulator framework (actually `TaskMetrics` and accumulators are on an awkward collision course as we add more features to the former). The current patch demo's how we can use the feature to provide instrumentation for RDD input sizes. The nice thing about going through accumulators is that users can read the current value of the data being tracked in their programs. This could be useful to e.g. decide to short-circuit a Spark stage depending on how things are going.
![counters](https://cloud.githubusercontent.com/assets/320616/3488815/6ee7bc34-0505-11e4-84ce-e36d9886e2cf.png)
Author: Patrick Wendell <pwendell@gmail.com>
Closes #1309 from pwendell/metrics and squashes the following commits:
8815308 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into HEAD
93fbe0f [Patrick Wendell] Other minor fixes
cc43f68 [Patrick Wendell] Updating unit tests
c991b1b [Patrick Wendell] Moving some code into the Accumulators class
9a9ba3c [Patrick Wendell] More merge fixes
c5ace9e [Patrick Wendell] More merge conflicts
1da15e3 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into metrics
9860c55 [Patrick Wendell] Potential solution to posting listener events
0bb0e33 [Patrick Wendell] Remove "display" variable and assume display = name.isDefined
0ec4ac7 [Patrick Wendell] Java API's
e95bf69 [Patrick Wendell] Stash
be97261 [Patrick Wendell] Style fix
8407308 [Patrick Wendell] Removing examples in Hadoop and RDD class
64d405f [Patrick Wendell] Adding missing file
5d8b156 [Patrick Wendell] Changes based on Kay's review.
9f18bad [Patrick Wendell] Minor style changes and tests
7a63abc [Patrick Wendell] Adding Json serialization and responding to Reynold's feedback
ad85076 [Patrick Wendell] Example of using named accumulators for custom RDD metrics.
0b72660 [Patrick Wendell] Initial WIP example of supporing globally named accumulators.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
It was easier to combine these 2 jira since they touch many of the same places. This pr adds the following:
- adds modify acls
- adds admin acls (list of admins/users that get added to both view and modify acls)
- modify Kill button on UI to take modify acls into account
- changes config name of spark.ui.acls.enable to spark.acls.enable since I choose poorly in original name. We keep backwards compatibility so people can still use spark.ui.acls.enable. The acls should apply to any web ui as well as any CLI interfaces.
- send view and modify acls information on to YARN so that YARN interfaces can use (yarn cli for killing applications for example).
Author: Thomas Graves <tgraves@apache.org>
Closes #1196 from tgravescs/SPARK-1890 and squashes the following commits:
8292eb1 [Thomas Graves] review comments
b92ec89 [Thomas Graves] remove unneeded variable from applistener
4c765f4 [Thomas Graves] Add in admin acls
72eb0ac [Thomas Graves] Add modify acls
(cherry picked from commit 1c5555a23d3aa40423d658cfbf2c956ad415a6b1)
Signed-off-by: Thomas Graves <tgraves@apache.org>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Author: wangfei <scnbwf@yeah.net>
Author: wangfei <wangfei1@huawei.com>
Closes #714 from scwf/memoryFraction and squashes the following commits:
6e385b9 [wangfei] Update SparkConf.scala
da6ee59 [wangfei] add configs
829a195 [wangfei] add indent
717c0ca [wangfei] updated to make more concise
fc45476 [wangfei] validate memoryfraction in sparkconf
2e79b3d [wangfei] && => ||
43621bd [wangfei] && => ||
cf38bcf [wangfei] throw IllegalArgumentException
14d18ac [wangfei] throw IllegalArgumentException
dff1f0f [wangfei] Update BlockManager.scala
764965f [wangfei] Update ExternalAppendOnlyMap.scala
a59d76b [wangfei] Throw exception when memoryFracton is out of range
7b899c2 [wangfei] 【SPARK-1779】
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
`master.ui.port` and `worker.ui.port` were never picked up by SparkConf, simply because they are not prefixed with "spark." Unfortunately, this is also currently the documented way of setting these values.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1779 from andrewor14/master-worker-port and squashes the following commits:
8475e95 [Andrew Or] Update docs to reflect changes in configs
4db3d5d [Andrew Or] Stop using configs that don't actually work
(cherry picked from commit a646a365e3beb8d0cd7e492e625ce68ee9439a07)
Signed-off-by: Patrick Wendell <pwendell@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
collections
This tracks memory properly if there are multiple spilling collections in the same task (which was a problem before), and also implements an algorithm that lets each thread grow up to 1 / 2N of the memory pool (where N is the number of threads) before spilling, which avoids an inefficiency with small spills we had before (some threads would spill many times at 0-1 MB because the pool was allocated elsewhere).
Author: Matei Zaharia <matei@databricks.com>
Closes #1707 from mateiz/spark-2711 and squashes the following commits:
debf75b [Matei Zaharia] Review comments
24f28f3 [Matei Zaharia] Small rename
c8f3a8b [Matei Zaharia] Update ShuffleMemoryManager to be able to partially grant requests
315e3a5 [Matei Zaharia] Some review comments
b810120 [Matei Zaharia] Create central manager to track memory for all spilling collections
(cherry picked from commit 4fde28c2063f673ec7f51d514ba62a73321960a1)
Signed-off-by: Matei Zaharia <matei@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Replaces this with an O(1) operation that does not have to shift over
the whole tail of the array into the gap produced by the element removed.
Author: Matei Zaharia <matei@databricks.com>
Closes #1773 from mateiz/SPARK-2685 and squashes the following commits:
1ea028a [Matei Zaharia] Update comments in StreamBuffer and EAOM, and reuse ArrayBuffers
eb1abfd [Matei Zaharia] Update ExternalAppendOnlyMap to avoid buffer.remove()
(cherry picked from commit 066765d60d21b6b9943862b788e4a4bd07396e6c)
Signed-off-by: Matei Zaharia <matei@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
SparkContext
Author: Reynold Xin <rxin@apache.org>
Closes #1772 from rxin/accumulator-dagscheduler and squashes the following commits:
6a58520 [Reynold Xin] [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext.
(cherry picked from commit 05bf4e4aff0d052a53d3e64c43688f07e27fec50)
Signed-off-by: Reynold Xin <rxin@apache.org>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
ExternalMap / Sorter
All these changes are from mridulm's work in #1609, but extracted here to fix this specific issue and make it easier to merge not 1.1. This particular set of changes is to make sure that we read exactly the right range of bytes from each spill file in EAOM: some serializers can write bytes after the last object (e.g. the TC_RESET flag in Java serialization) and that would confuse the previous code into reading it as part of the next batch. There are also improvements to cleanup to make sure files are closed.
In addition to bringing in the changes to ExternalAppendOnlyMap, I also copied them to the corresponding code in ExternalSorter and updated its test suite to test for the same issues.
Author: Matei Zaharia <matei@databricks.com>
Closes #1722 from mateiz/spark-2792 and squashes the following commits:
5d4bfb5 [Matei Zaharia] Make objectStreamReset counter count the last object written too
18fe865 [Matei Zaharia] Update docs on objectStreamReset
576ee83 [Matei Zaharia] Allow objectStreamReset to be 0
0374217 [Matei Zaharia] Remove super paranoid code to close file handles
bda37bb [Matei Zaharia] Implement Mridul's ExternalAppendOnlyMap fixes in ExternalSorter too
0d6dad7 [Matei Zaharia] Added Mridul's test changes for ExternalAppendOnlyMap
9a78e4b [Matei Zaharia] Add @mridulm's fixes to ExternalAppendOnlyMap for batch sizes
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Kill only the python worker related to cancelled tasks.
The daemon will start a background thread to monitor all the opened sockets for all workers. If the socket is closed by JVM, this thread will kill the worker.
When an task is cancelled, the socket to worker will be closed, then the worker will be killed by deamon.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1643 from davies/kill and squashes the following commits:
8ffe9f3 [Davies Liu] kill worker by deamon, because runtime.exec() is too heavy
46ca150 [Davies Liu] address comment
acd751c [Davies Liu] kill the worker when task is canceled
(cherry picked from commit 55349f9fe81ba5af5e4a5e4908ebf174e63c6cc9)
Signed-off-by: Josh Rosen <joshrosen@apache.org>
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Minor fixes on top of #1679.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1736 from andrewor14/amend-#1679 and squashes the following commits:
3b46f5e [Andrew Or] Minor fixes
(cherry picked from commit 3dc55fdf450b4237f7c592fce56d1467fd206366)
Signed-off-by: Patrick Wendell <pwendell@gmail.com>
|
|
|
|
|
|
|
|
|
|
| |
This also Closes #1701.
Author: GuoQiang Li <witgo@qq.com>
Closes #1208 from witgo/SPARK-1470 and squashes the following commits:
422646b [GuoQiang Li] Remove scalalogging-slf4j dependency
|
|
|
|
|
|
|
|
|
|
| |
`spark.test.home` was introduced in #1734. This is fine for SBT but is failing maven tests. Either way it shouldn't throw an NPE.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1739 from andrewor14/fix-spark-test-home and squashes the following commits:
ce2624c [Andrew Or] Do not throw NPE if spark.test.home is not set
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
When standalone Workers launch executors, they inherit the Spark home set by the driver. This means if the worker machines do not share the same directory structure as the driver node, the Workers will attempt to run scripts (e.g. bin/compute-classpath.sh) that do not exist locally and fail. This is a common scenario if the driver is launched from outside of the cluster.
The solution is to simply not pass the driver's Spark home to the Workers. This PR further makes an attempt to avoid overloading the usages of `spark.home`, which is now only used for setting executor Spark home on Mesos and in python.
This is based on top of #1392 and originally reported by YanTangZhai. Tested on standalone cluster.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1734 from andrewor14/spark-home-reprise and squashes the following commits:
f71f391 [Andrew Or] Revert changes in python
1c2532c [Andrew Or] Merge branch 'master' of github.com:apache/spark into spark-home-reprise
188fc5d [Andrew Or] Avoid using spark.home where possible
09272b7 [Andrew Or] Always use Worker's working directory as spark home
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The existing code in `StorageUtils` is not the most efficient. Every time we want to update an `RDDInfo` we end up iterating through all blocks on all block managers just to discard most of them. The symptoms manifest themselves in the bountiful UI bugs observed in the wild. Many of these bugs are caused by the slow consumption of events in `LiveListenerBus`, which frequently leads to the event queue overflowing and `SparkListenerEvent`s being dropped on the floor. The changes made in this PR avoid this by first filtering out only the blocks relevant to us before computing storage information from them.
It's worth a mention that this corner of the Spark code is also not very well-tested at all. The bulk of the changes in this PR (more than 60%) is actually test cases for the various logic in `StorageUtils.scala` as well as `StorageTab.scala`. These will eventually be extended to cover the various listeners that constitute the `SparkUI`.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1679 from andrewor14/fix-drop-events and squashes the following commits:
f80c1fa [Andrew Or] Rewrite fold and reduceOption as sum
e132d69 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
14fa1c3 [Andrew Or] Simplify some code + update a few comments
a91be46 [Andrew Or] Make ExecutorsPage blazingly fast
bf6f09b [Andrew Or] Minor changes
8981de1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
af19bc0 [Andrew Or] *UsedByRDD -> *UsedByRdd (minor)
6970bc8 [Andrew Or] Add extensive tests for StorageListener and the new code in StorageUtils
e080b9e [Andrew Or] Reduce run time of StorageUtils.updateRddInfo to near constant
2c3ef6a [Andrew Or] Actually filter out only the relevant RDDs
6fef86a [Andrew Or] Add extensive tests for new code in StorageStatus
b66b6b0 [Andrew Or] Use more efficient underlying data structures for blocks
6a7b7c0 [Andrew Or] Avoid chained operations on TraversableLike
a9ec384 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
b12fcd7 [Andrew Or] Fix tests + simplify sc.getRDDStorageInfo
da8e322 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events
8e91921 [Andrew Or] Iterate through a filtered set of blocks when updating RDDInfo
7b2c4aa [Andrew Or] Rewrite blockLocationsFromStorageStatus + clean up method signatures
41fa50d [Andrew Or] Add a legacy constructor for StorageStatus
53af15d [Andrew Or] Refactor StorageStatus + add a bunch of tests
|
|
|
|
|
|
| |
the directly sfl4j api"
This reverts commit adc8303294e26efb4ed15e5f5ba1062f7988625d.
|
|
|
|
|
|
|
|
|
|
|
| |
directly sfl4j api
Author: GuoQiang Li <witgo@qq.com>
Closes #1369 from witgo/SPARK-1470_new and squashes the following commits:
66a1641 [GuoQiang Li] IncompatibleResultTypeProblem
73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Curently, daemon.py forks a pool of numProcessors subprocesses, and those processes fork themselves again to create the actual Python worker processes that handle data.
I think that this extra layer of indirection is unnecessary and adds a lot of complexity. This commit attempts to remove this middle layer of subprocesses by launching the workers directly from daemon.py.
See https://github.com/mesos/spark/pull/563 for the original PR that added daemon.py, where I raise some issues with the current design.
Author: Josh Rosen <joshrosen@apache.org>
Closes #1680 from JoshRosen/pyspark-daemon and squashes the following commits:
5abbcb9 [Josh Rosen] Replace magic number: 4 -> EINTR
5495dff [Josh Rosen] Throw IllegalStateException if worker launch fails.
b79254d [Josh Rosen] Detect failed fork() calls; improve error logging.
282c2c4 [Josh Rosen] Remove daemon.py exit logging, since it caused problems:
8554536 [Josh Rosen] Fix daemon’s shutdown(); log shutdown reason.
4e0fab8 [Josh Rosen] Remove shared-memory exit_flag; don't die on worker death.
e9892b4 [Josh Rosen] [WIP] [SPARK-2764] Simplify daemon.py process structure.
|
|
|
|
|
|
|
|
|
|
| |
If SPARK_CONF_DIR environment variable is set, search it for spark-defaults.conf.
Author: Albert Chu <chu11@llnl.gov>
Closes #1059 from chu11/SPARK-2116 and squashes the following commits:
9f3ac94 [Albert Chu] SPARK-2116: If SPARK_CONF_DIR environment variable is set, search it for spark-defaults.conf.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.
This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.
root
|-- field1: integer (nullable = true)
|-- field2: string (nullable = true)
|-- field3: struct (nullable = true)
| |-- field4: integer (nullable = true)
| |-- field5: array (nullable = true)
| | |-- element: integer (containsNull = false)
|-- field6: array (nullable = true)
| |-- element: struct (containsNull = false)
| | |-- field7: string (nullable = true)
Then we can access them by row.field3.field5[0] or row.field6[5].field7
It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.
You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:
ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))
Or you could use Row to create a class just like namedtuple, for example:
Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))
Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.
schema = StructType([StructField("name, StringType, True),
StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)
PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1598 from davies/nested and squashes the following commits:
f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
consolidation
All changes from this PR are by mridulm and are drawn from his work in #1609. This patch is intended to fix all major issues related to shuffle file consolidation that mridulm found, while minimizing changes to the code, with the hope that it may be more easily merged into 1.1.
This patch is **not** intended as a replacement for #1609, which provides many additional benefits, including fixes to ExternalAppendOnlyMap, improvements to DiskBlockObjectWriter's API, and several new unit tests.
If it is feasible to merge #1609 for the 1.1 deadline, that is a preferable option.
Author: Aaron Davidson <aaron@databricks.com>
Closes #1678 from aarondav/consol and squashes the following commits:
53b3f6d [Aaron Davidson] Correct behavior when writing unopened file
701d045 [Aaron Davidson] Rebase with sort-based shuffle
9160149 [Aaron Davidson] SPARK-2532: Minimal shuffle consolidation fixes
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
JIRA: https://issues.apache.org/jira/browse/SPARK-1612
Move the "close" statements into a "finally" block.
Author: zsxwing <zsxwing@gmail.com>
Closes #535 from zsxwing/SPARK-1612 and squashes the following commits:
ae52f50 [zsxwing] Update to follow the code style
549ba13 [zsxwing] SPARK-1612: Fix potential resource leaks
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
When performing some transformations on RDDs after many iterations, the dependencies of RDDs could be very long. It can easily cause StackOverflowError when recursively visiting these dependencies in Spark core. For example:
var rdd = sc.makeRDD(Array(1))
for (i <- 1 to 1000) {
rdd = rdd.coalesce(1).cache()
rdd.collect()
}
This PR changes recursive visiting on rdd's dependencies to iterative approach to avoid StackOverflowError.
In addition to the recursive visiting, since the Java serializer has a known [bug](http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4152790) that causes StackOverflowError too when serializing/deserializing a large graph of objects. So applying this PR only solves part of the problem. Using KryoSerializer to replace Java serializer might be helpful. However, since KryoSerializer is not supported for `spark.closure.serializer` now, I can not test if KryoSerializer can solve Java serializer's problem completely.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes #1418 from viirya/remove_recursive_visit and squashes the following commits:
6b2c615 [Liang-Chi Hsieh] change function name; comply with code style.
5f072a7 [Liang-Chi Hsieh] add comments to explain Stack usage.
8742dbb [Liang-Chi Hsieh] comply with code style.
900538b [Liang-Chi Hsieh] change recursive visiting on rdd's dependencies to iterative approach to avoid stackoverflowerror.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
getPreferredLocs traverses a dependency graph of partitions using depth first search. Given a complex dependency graph, the old implementation may explore a set of paths in the graph that is exponential in the number of nodes. By maintaining a set of visited nodes the new implementation avoids revisiting nodes, preventing exponential blowup.
Some comment and whitespace cleanups are also included.
Author: Aaron Staple <aaron.staple@gmail.com>
Closes #1362 from staple/SPARK-695 and squashes the following commits:
ecea0f3 [Aaron Staple] address review comments
751c661 [Aaron Staple] [SPARK-695] Add a unit test.
5adf326 [Aaron Staple] Replace getPreferredLocsInternal's HashMap argument with a simpler HashSet.
58e37d0 [Aaron Staple] Replace comment documenting NarrowDependency.
6751ced [Aaron Staple] Revert "Remove unused variable."
04c7097 [Aaron Staple] Fix indentation.
0030884 [Aaron Staple] Remove unused variable.
33f67c6 [Aaron Staple] Clarify comment.
4e42b46 [Aaron Staple] Remove apparently incorrect comment describing NarrowDependency.
65c2d3d [Aaron Staple] [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This is a sketch of a patch that allows the UI to show metrics for tasks that have not yet completed. It adds a heartbeat every 2 seconds from the executors to the driver, reporting metrics for all of the executor's tasks.
It still needs unit tests, polish, and cluster testing, but I wanted to put it up to get feedback on the approach.
Author: Sandy Ryza <sandy@cloudera.com>
Closes #1056 from sryza/sandy-spark-2099 and squashes the following commits:
93b9fdb [Sandy Ryza] Up heartbeat interval to 10 seconds and other tidying
132aec7 [Sandy Ryza] Heartbeat and HeartbeatResponse are already Serializable as case classes
38dffde [Sandy Ryza] Additional review feedback and restore test that was removed in BlockManagerSuite
51fa396 [Sandy Ryza] Remove hostname race, add better comments about threading, and some stylistic improvements
3084f10 [Sandy Ryza] Make TaskUIData a case class again
3bda974 [Sandy Ryza] Stylistic fixes
0dae734 [Sandy Ryza] SPARK-2099. Report progress while task is running.
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
local-n-failures consistent
[SPARK-2557](https://issues.apache.org/jira/browse/SPARK-2557)
Author: Ye Xianjin <advancedxy@gmail.com>
Closes #1464 from advancedxy/SPARK-2557 and squashes the following commits:
d844d67 [Ye Xianjin] add local-*-n-failures, bad-local-n, bad-local-n-failures test case
3bbc668 [Ye Xianjin] fix LOCAL_N_REGEX regular expression and make local_n_failures accept * as all cores on the computer
|
|
|
|
|
|
|
|
| |
Author: Rahul Singhal <rahul.singhal@guavus.com>
Closes #1076 from rahulsinghaliitd/SPARK-2134 and squashes the following commits:
15f18b6 [Rahul Singhal] SPARK-2134: Report metrics before application finishes
|
|
|
|
|
|
|
|
|
|
|
|
| |
This patch simply uses the ExternalSorter class from sort-based shuffle.
Closes #931 and Closes #1090
Author: Matei Zaharia <matei@databricks.com>
Closes #1677 from mateiz/spark-983 and squashes the following commits:
96b3fda [Matei Zaharia] SPARK-983. Support external sorting in sortByKey()
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Closes #1578 from sarutak/SPARK-2670 and squashes the following commits:
85c8938 [Kousuke Saruta] Removed useless results.put for fail fast
e8713cc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
d353984 [Kousuke Saruta] Refined assertion messages in BlockFetcherIteratorSuite.scala
03bcb02 [Kousuke Saruta] Merge branch 'SPARK-2670' of github.com:sarutak/spark into SPARK-2670
5d05855 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
4fca130 [Kousuke Saruta] Added test cases for BasicBlockFetcherIterator
b7b8250 [Kousuke Saruta] Modified BasicBlockFetchIterator to fail fast when local fetch error has been occurred
a3a9be1 [Kousuke Saruta] Modified BlockFetcherIterator for SPARK-2670
460dc01 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670
e310c0b [Kousuke Saruta] Modified BlockFetcherIterator to handle local fetch failure as fatch fail
|
|
|
|
|
|
|
|
| |
Author: Sandy Ryza <sandy@cloudera.com>
Closes #1642 from sryza/sandy-spark-2738 and squashes the following commits:
a923e4e [Sandy Ryza] SPARK-2738. Remove redundant imports in BlockManagerSuite
|
|
|
|
|
|
|
|
| |
Author: Haoyuan Li <haoyuan@cs.berkeley.edu>
Closes #1651 from haoyuan/upgrade-tachyon and squashes the following commits:
6f3f98f [Haoyuan Li] upgrade tachyon to 0.5.0
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Prior to this change, every PySpark task completion opened a new socket to the accumulator server, passed its updates through, and then quit. I'm not entirely sure why PySpark always sends accumulator updates, but regardless this causes a very rapid buildup of ephemeral TCP connections that remain in the TCP_WAIT state for around a minute before being cleaned up.
Rather than trying to allow these sockets to be cleaned up faster, this patch simply reuses the connection between tasks completions (since they're fed updates in a single-threaded manner by the DAGScheduler anyway).
The only tricky part here was making sure that the AccumulatorServer was able to shutdown in a timely manner (i.e., stop polling for new data), and this was accomplished via minor feats of magic.
I have confirmed that this patch eliminates the buildup of ephemeral sockets due to the accumulator updates. However, I did note that there were still significant sockets being created against the PySpark daemon port, but my machine was not able to create enough sockets fast enough to fail. This may not be the last time we've seen this issue, though.
Author: Aaron Davidson <aaron@databricks.com>
Closes #1503 from aarondav/accum and squashes the following commits:
b3e12f7 [Aaron Davidson] SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark
|
|
|
|
|
|
|
|
|
|
| |
It should be more convenient if user can specify ascending and numPartitions when calling sortByKey.
Author: Rui Li <rui.li@intel.com>
Closes #1645 from lirui-intel/spark-2740 and squashes the following commits:
fb5d52e [Rui Li] SPARK-2740: allow user to specify ascending and numPartitions for sortByKey
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The logging code that handles log4j initialization leads to an stack overflow error when used with log4j 2.x, which has just been released. This occurs even a downstream project has correctly adjusted SLF4J bindings, and that is the right thing to do for log4j 2.x, since it is effectively a separate project from 1.x.
Here is the relevant bit of Logging.scala:
```
private def initializeLogging() {
// If Log4j is being used, but is not initialized, load a default properties file
val binder = StaticLoggerBinder.getSingleton
val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory")
val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
if (!log4jInitialized && usingLog4j) {
val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match {
case Some(url) =>
PropertyConfigurator.configure(url)
log.info(s"Using Spark's default log4j profile: $defaultLogProps")
case None =>
System.err.println(s"Spark was unable to load $defaultLogProps")
}
}
Logging.initialized = true
// Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
// and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
log
}
```
The first minor issue is that there is a call to a logger inside this method, which is initializing logging. In this situation, it ends up causing the initialization to be called recursively until the stack overflow. It would be slightly tidier to log this only after Logging.initialized = true. Or not at all. But it's not the root problem, or else, it would not work at all now.
The calls to log4j classes here always reference log4j 1.2 no matter what. For example, there is not getAllAppenders in log4j 2.x. That's fine. Really, "usingLog4j" means "using log4j 1.2" and "log4jInitialized" means "log4j 1.2 is initialized".
usingLog4j should be false for log4j 2.x, because the initialization only matters for log4j 1.2. But, it's true, and that's the real issue. And log4jInitialized is always false, since calls to the log4j 1.2 API are stubs and no-ops in this setup, where the caller has swapped in log4j 2.x. Hence the loop.
This is fixed, I believe, if "usingLog4j" can be false for log4j 2.x. The SLF4J static binding class has the same name for both versions, unfortunately, which causes the issue. However they're in different packages. For example, if the test included "... and begins with org.slf4j", it should work, as the SLF4J binding for log4j 2.x is provided by log4j 2.x at the moment, and is in package org.apache.logging.slf4j.
Of course, I assume that SLF4J will eventually offer its own binding. I hope to goodness they at least name the binding class differently, or else this will again not work. But then some other check can probably be made.
Author: Sean Owen <srowen@gmail.com>
Closes #1547 from srowen/SPARK-2646 and squashes the following commits:
92a9898 [Sean Owen] System.out -> System.err
94be4c7 [Sean Owen] Add back log message as System.out, with informational comment
a7f8876 [Sean Owen] Updates from review
6f3c1d3 [Sean Owen] Remove log statement in logging initialization, and distinguish log4j 1.2 from 2.0, to avoid stack overflow in initialization
|
|
|
|
|
|
|
|
|
|
| |
...ags
Author: Sandy Ryza <sandy@cloudera.com>
Closes #1665 from sryza/sandy-spark-2664 and squashes the following commits:
0518c63 [Sandy Ryza] SPARK-2664. Deal with `--conf` options in spark-submit that relate to flags
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This allows users to gain access to the InputSplit which backs each partition.
An alternative solution would have been to have a .withInputSplit() method which returns a new RDD[(InputSplit, (K, V))], but this is confusing because you could not cache this RDD or shuffle it, as InputSplit is not inherently serializable.
Author: Aaron Davidson <aaron@databricks.com>
Closes #973 from aarondav/hadoop and squashes the following commits:
9c9112b [Aaron Davidson] Add JavaAPISuite test
9942cd7 [Aaron Davidson] Add Java API
1284a3a [Aaron Davidson] SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The Java API's use of fake ClassTags doesn't seem to cause any problems for Java users, but it can lead to issues when passing JavaRDDs' underlying RDDs to Scala code (e.g. in the MLlib Java API wrapper code). If we call collect() on a Scala RDD with an incorrect ClassTag, this causes ClassCastExceptions when we try to allocate an array of the wrong type (for example, see SPARK-2197).
There are a few possible fixes here. An API-breaking fix would be to completely remove the fake ClassTags and require Java API users to pass java.lang.Class instances to all parallelize() calls and add returnClass fields to all Function implementations. This would be extremely verbose.
Instead, this patch adds internal APIs to "repair" a Scala RDD with an incorrect ClassTag by wrapping it and overriding its ClassTag. This should be okay for cases where the Scala code that calls collect() knows what type of array should be allocated, which is the case in the MLlib wrappers.
Author: Josh Rosen <joshrosen@apache.org>
Closes #1639 from JoshRosen/SPARK-2737 and squashes the following commits:
572b4c8 [Josh Rosen] Replace newRDD[T] with mapPartitions().
469d941 [Josh Rosen] Preserve partitioner in retag().
af78816 [Josh Rosen] Allow retag() to get classTag implicitly.
d1d54e6 [Josh Rosen] [SPARK-2737] Add retag() method for changing RDDs' ClassTags.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
We resolve relative paths to the local `file:/` system for `--jars` and `--files` in spark submit (#853). We should do the same for the history server.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1280 from andrewor14/hist-serv-fix and squashes the following commits:
13ff406 [Andrew Or] Merge branch 'master' of github.com:apache/spark into hist-serv-fix
b393e17 [Andrew Or] Strip trailing "/" from logging directory
622a471 [Andrew Or] Fix test in EventLoggingListenerSuite
0e20f71 [Andrew Or] Shift responsibility of resolving paths up one level
b037c0c [Andrew Or] Use resolved paths for everything in history server
c7e36ee [Andrew Or] Resolve paths for event logging too
40e3933 [Andrew Or] Resolve history server file paths
|
|
|
|
|
|
|
|
|
| |
Author: Reynold Xin <rxin@apache.org>
Closes #1675 from rxin/unionrdd and squashes the following commits:
941d316 [Reynold Xin] Clear RDDs for checkpointing.
c9f05f2 [Reynold Xin] [SPARK-2758] UnionRDD's UnionPartition should not reference parent RDDs
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This adds a new ShuffleManager based on sorting, as described in https://issues.apache.org/jira/browse/SPARK-2045. The bulk of the code is in an ExternalSorter class that is similar to ExternalAppendOnlyMap, but sorts key-value pairs by partition ID and can be used to create a single sorted file with a map task's output. (Longer-term I think this can take on the remaining functionality in ExternalAppendOnlyMap and replace it so we don't have code duplication.)
The main TODOs still left are:
- [x] enabling ExternalSorter to merge across spilled files
- [x] with an Ordering
- [x] without an Ordering, using the keys' hash codes
- [x] adding more tests (e.g. a version of our shuffle suite that runs on this)
- [x] rebasing on top of the size-tracking refactoring in #1165 when that is merged
- [x] disabling spilling if spark.shuffle.spill is set to false
Despite this though, this seems to work pretty well (running successfully in cases where the hash shuffle would OOM, such as 1000 reduce tasks on executors with only 1G memory), and it seems to be comparable in speed or faster than hash-based shuffle (it will create much fewer files for the OS to keep track of). So I'm posting it to get some early feedback.
After these TODOs are done, I'd also like to enable ExternalSorter to sort data within each partition by a key as well, which will allow us to use it to implement external spilling in reduce tasks in `sortByKey`.
Author: Matei Zaharia <matei@databricks.com>
Closes #1499 from mateiz/sort-based-shuffle and squashes the following commits:
bd841f9 [Matei Zaharia] Various review comments
d1c137fd [Matei Zaharia] Various review comments
a611159 [Matei Zaharia] Compile fixes due to rebase
62c56c8 [Matei Zaharia] Fix ShuffledRDD sometimes not returning Tuple2s.
f617432 [Matei Zaharia] Fix a failing test (seems to be due to change in SizeTracker logic)
9464d5f [Matei Zaharia] Simplify code and fix conflicts after latest rebase
0174149 [Matei Zaharia] Add cleanup behavior and cleanup tests for sort-based shuffle
eb4ee0d [Matei Zaharia] Remove customizable element type in ShuffledRDD
fa2e8db [Matei Zaharia] Allow nextBatchStream to be called after we're done looking at all streams
a34b352 [Matei Zaharia] Fix tracking of indices within a partition in SpillReader, and add test
03e1006 [Matei Zaharia] Add a SortShuffleSuite that runs ShuffleSuite with sort-based shuffle
3c7ff1f [Matei Zaharia] Obey the spark.shuffle.spill setting in ExternalSorter
ad65fbd [Matei Zaharia] Rebase on top of Aaron's Sorter change, and use Sorter in our buffer
44d2a93 [Matei Zaharia] Use estimateSize instead of atGrowThreshold to test collection sizes
5686f71 [Matei Zaharia] Optimize merging phase for in-memory only data:
5461cbb [Matei Zaharia] Review comments and more tests (e.g. tests with 1 element per partition)
e9ad356 [Matei Zaharia] Update ContextCleanerSuite to make sure shuffle cleanup tests use hash shuffle (since they were written for it)
c72362a [Matei Zaharia] Added bug fix and test for when iterators are empty
de1fb40 [Matei Zaharia] Make trait SizeTrackingCollection private[spark]
4988d16 [Matei Zaharia] tweak
c1b7572 [Matei Zaharia] Small optimization
ba7db7f [Matei Zaharia] Handle null keys in hash-based comparator, and add tests for collisions
ef4e397 [Matei Zaharia] Support for partial aggregation even without an Ordering
4b7a5ce [Matei Zaharia] More tests, and ability to sort data if a total ordering is given
e1f84be [Matei Zaharia] Fix disk block manager test
5a40a1c [Matei Zaharia] More tests
614f1b4 [Matei Zaharia] Add spill metrics to map tasks
cc52caf [Matei Zaharia] Add more error handling and tests for error cases
bbf359d [Matei Zaharia] More work
3a56341 [Matei Zaharia] More partial work towards sort-based shuffle
7a0895d [Matei Zaharia] Some more partial work towards sort-based shuffle
b615476 [Matei Zaharia] Scaffolding for sort-based shuffle
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
builds; missing junit:junit dep
The Maven-based builds in the build matrix have been failing for a few days:
https://amplab.cs.berkeley.edu/jenkins/view/Spark/
On inspection, it looks like the Spark SQL Java tests don't compile:
https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/244/consoleFull
I confirmed it by repeating the command vs master:
`mvn -Dhadoop.version=1.0.4 -Dlabel=centos -DskipTests clean package`
The problem is that this module doesn't depend on JUnit. In fact, none of the modules do, but `com.novocode:junit-interface` (the SBT-JUnit bridge) pulls it in, in most places. However this module doesn't depend on `com.novocode:junit-interface`
Adding the `junit:junit` dependency fixes the compile problem. In fact, the other modules with Java tests should probably depend on it explicitly instead of happening to get it via `com.novocode:junit-interface`, since that is a bit SBT/Scala-specific (and I am not even sure it's needed).
Author: Sean Owen <srowen@gmail.com>
Closes #1660 from srowen/SPARK-2749 and squashes the following commits:
858ff7c [Sean Owen] Add explicit junit dep to other modules with Java tests for robustness
9636794 [Sean Owen] Add junit dep so that Spark SQL Java tests compile
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024
This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats.
* Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs.
* Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types.
* No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples.
* Added HBase and Cassandra output examples to show how custom output formats and converters can be used.
cc MLnick mateiz ahirreddy pwendell
Author: Kan Zhang <kzhang@apache.org>
Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits:
c01e3ef [Kan Zhang] [SPARK-2024] code formatting
6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD
d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10
57a7a5e [Kan Zhang] [SPARK-2024] correcting typo
75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD
0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests
9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests
0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases
7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This is a resubmission of #1452. It was reverted because it broke the build.
Currently (as of Spark 1.0.1), Spark sends RDD object (which contains closures) using Akka along with the task itself to the executors. This is inefficient because all tasks in the same stage use the same RDD object, but we have to send RDD object multiple times to the executors. This is especially bad when a closure references some variable that is very large. The current design led to users having to explicitly broadcast large variables.
The patch uses broadcast to send RDD objects and the closures to executors, and use Akka to only send a reference to the broadcast RDD/closure along with the partition specific information for the task. For those of you who know more about the internals, Spark already relies on broadcast to send the Hadoop JobConf every time it uses the Hadoop input, because the JobConf is large.
The user-facing impact of the change include:
1. Users won't need to decide what to broadcast anymore, unless they would want to use a large object multiple times in different operations
2. Task size will get smaller, resulting in faster scheduling and higher task dispatch throughput.
In addition, the change will simplify some internals of Spark, eliminating the need to maintain task caches and the complex logic to broadcast JobConf (which also led to a deadlock recently).
A simple way to test this:
```scala
val a = new Array[Byte](1000*1000); scala.util.Random.nextBytes(a);
sc.parallelize(1 to 1000, 1000).map { x => a; x }.groupBy { x => a; x }.count
```
Numbers on 3 r3.8xlarge instances on EC2
```
master branch: 5.648436068 s, 4.715361895 s, 5.360161877 s
with this change: 3.416348793 s, 1.477846558 s, 1.553432156 s
```
Author: Reynold Xin <rxin@apache.org>
Closes #1498 from rxin/broadcast-task and squashes the following commits:
f7364db [Reynold Xin] Code review feedback.
f8535dc [Reynold Xin] Fixed the style violation.
252238d [Reynold Xin] Serialize the final task closure as well as ShuffleDependency in taskBinary.
111007d [Reynold Xin] Fix broadcast tests.
797c247 [Reynold Xin] Properly send SparkListenerStageSubmitted and SparkListenerStageCompleted.
bab1d8b [Reynold Xin] Check for NotSerializableException in submitMissingTasks.
cf38450 [Reynold Xin] Use TorrentBroadcastFactory.
991c002 [Reynold Xin] Use HttpBroadcast.
de779f8 [Reynold Xin] Fix TaskContextSuite.
cc152fc [Reynold Xin] Don't cache the RDD broadcast variable.
d256b45 [Reynold Xin] Fixed unit test failures. One more to go.
cae0af3 [Reynold Xin] [SPARK-2521] Broadcast RDD object (instead of sending it along with every task).
|
|
|
|
|
|
|
|
|
|
|
|
| |
Author: Koert Kuipers <koert@tresata.com>
Closes #735 from koertkuipers/feat-kryo-max-buffersize and squashes the following commits:
15f6d81 [Koert Kuipers] change default for spark.kryoserializer.buffer.max.mb to 64mb and add some documentation
1bcc22c [Koert Kuipers] Merge branch 'master' into feat-kryo-max-buffersize
0c9f8eb [Koert Kuipers] make default for kryo max buffer size 16MB
143ec4d [Koert Kuipers] test resizable buffer in kryo Output
0732445 [Koert Kuipers] support setting maxCapacity to something different than capacity in kryo Output
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The current PR contains the following changes:
* Expose `DataType`s in the sql package (internal details are private to sql).
* Users can create Rows.
* Introduce `applySchema` to create a `SchemaRDD` by applying a `schema: StructType` to an `RDD[Row]`.
* Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`.
* `ScalaReflection.typeOfObject` provides a way to infer the Catalyst data type based on an object. Also, we can compose `typeOfObject` with some custom logics to form a new function to infer the data type (for different use cases).
* `JsonRDD` has been refactored to use changes introduced by this PR.
* Add a field `containsNull` to `ArrayType`. So, we can explicitly mark if an `ArrayType` can contain null values. The default value of `containsNull` is `false`.
New APIs are introduced in the sql package object and SQLContext. You can find the scaladoc at
[sql package object](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.package) and [SQLContext](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.SQLContext).
An example of using `applySchema` is shown below.
```scala
import org.apache.spark.sql._
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
val schema =
StructType(
StructField("name", StringType, false) ::
StructField("age", IntegerType, true) :: Nil)
val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Row(p(0), p(1).trim.toInt))
val peopleSchemaRDD = sqlContext. applySchema(people, schema)
peopleSchemaRDD.printSchema
// root
// |-- name: string (nullable = false)
// |-- age: integer (nullable = true)
peopleSchemaRDD.registerAsTable("people")
sqlContext.sql("select name from people").collect.foreach(println)
```
I will add new contents to the SQL programming guide later.
JIRA: https://issues.apache.org/jira/browse/SPARK-2179
Author: Yin Huai <huai@cse.ohio-state.edu>
Closes #1346 from yhuai/dataTypeAndSchema and squashes the following commits:
1d45977 [Yin Huai] Clean up.
a6e08b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
c712fbf [Yin Huai] Converts types of values based on defined schema.
4ceeb66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
e5f8df5 [Yin Huai] Scaladoc.
122d1e7 [Yin Huai] Address comments.
03bfd95 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
2476ed0 [Yin Huai] Minor updates.
ab71f21 [Yin Huai] Format.
fc2bed1 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
bd40a33 [Yin Huai] Address comments.
991f860 [Yin Huai] Move "asJavaDataType" and "asScalaDataType" to DataTypeConversions.scala.
1cb35fe [Yin Huai] Add "valueContainsNull" to MapType.
3edb3ae [Yin Huai] Python doc.
692c0b9 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
1d93395 [Yin Huai] Python APIs.
246da96 [Yin Huai] Add java data type APIs to javadoc index.
1db9531 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
d48fc7b [Yin Huai] Minor updates.
33c4fec [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
b9f3071 [Yin Huai] Java API for applySchema.
1c9f33c [Yin Huai] Java APIs for DataTypes and Row.
624765c [Yin Huai] Tests for applySchema.
aa92e84 [Yin Huai] Update data type tests.
8da1a17 [Yin Huai] Add Row.fromSeq.
9c99bc0 [Yin Huai] Several minor updates.
1d9c13a [Yin Huai] Update applySchema API.
85e9b51 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
e495e4e [Yin Huai] More comments.
42d47a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
c3f4a02 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
2e58dbd [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
b8b7db4 [Yin Huai] 1. Move sql package object and package-info to sql-core. 2. Minor updates on APIs. 3. Update scala doc.
68525a2 [Yin Huai] Update JSON unit test.
3209108 [Yin Huai] Add unit tests.
dcaf22f [Yin Huai] Add a field containsNull to ArrayType to indicate if an array can contain null values or not. If an ArrayType is constructed by "ArrayType(elementType)" (the existing constructor), the value of containsNull is false.
9168b83 [Yin Huai] Update comments.
fc649d7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
eca7d04 [Yin Huai] Add two apply methods which will be used to extract StructField(s) from a StructType.
949d6bb [Yin Huai] When creating a SchemaRDD for a JSON dataset, users can apply an existing schema.
7a6a7e5 [Yin Huai] Fix bug introduced by the change made on SQLContext.inferSchema.
43a45e1 [Yin Huai] Remove sql.util.package introduced in a previous commit.
0266761 [Yin Huai] Format
03eec4c [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
90460ac [Yin Huai] Infer the Catalyst data type from an object and cast a data value to the expected type.
3fa0df5 [Yin Huai] Provide easier ways to construct a StructType.
16be3e5 [Yin Huai] This commit contains three changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Introduce `createSchemaRDD` to create a `SchemaRDD` from an `RDD` with a provided schema (represented by a `StructType`) and a provided function to construct `Row`, * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The main thing was that spark configs were not propagated to the driver, and so applications that do not specify `master` or `appName` automatically failed. This PR fixes that and a couple of miscellaneous things that are related.
One thing that may or may not be an issue is that the jars must be available on the driver node. In `standalone-cluster` mode, this effectively means these jars must be available on all the worker machines, since the driver is launched on one of them. The semantics here are not the same as `yarn-cluster` mode, where all the relevant jars are uploaded to a distributed cache automatically and shipped to the containers. This is probably not a concern, but still worth a mention.
Author: Andrew Or <andrewor14@gmail.com>
Closes #1538 from andrewor14/standalone-cluster and squashes the following commits:
8c11a0d [Andrew Or] Clean up imports / comments (minor)
2678d13 [Andrew Or] Handle extraJavaOpts properly
7660547 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster
6f64a9b [Andrew Or] Revert changes in YARN
2f2908b [Andrew Or] Fix tests
ed01491 [Andrew Or] Don't go overboard with escaping
8e105e1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster
b890949 [Andrew Or] Abstract usages of converting spark opts to java opts
79f63a3 [Andrew Or] Move sparkProps into javaOpts
78752f8 [Andrew Or] Fix tests
5a9c6c7 [Andrew Or] Fix line too long
c141a00 [Andrew Or] Don't display "unknown app" on driver log pages
d7e2728 [Andrew Or] Avoid deprecation warning in standalone Client
6ceb14f [Andrew Or] Allow relevant configs to propagate to standalone Driver
7f854bc [Andrew Or] Fix test
855256e [Andrew Or] Fix standalone-cluster mode
fd9da51 [Andrew Or] Formatting changes (minor)
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
As of Spark 1.0, RangePartitioner goes through data twice: once to compute the count and once to do sampling. As a result, to do sortByKey, Spark goes through data 3 times (once to count, once to sample, and once to sort).
`RangePartitioner` should go through data only once, collecting samples from input partitions as well as counting. If the data is balanced, this should give us a good sketch. If we see big partitions, we re-sample from them in order to collect enough items.
The downside is that we need to collect more from each partition in the first pass. An alternative solution is caching the intermediate result and decide whether to fetch the data after.
Author: Xiangrui Meng <meng@databricks.com>
Author: Reynold Xin <rxin@apache.org>
Closes #1562 from mengxr/range-partitioner and squashes the following commits:
6cc2551 [Xiangrui Meng] change foreach to for
eb39b08 [Xiangrui Meng] Merge branch 'master' into range-partitioner
eb95dd8 [Xiangrui Meng] separate sketching and determining bounds impl
c436d30 [Xiangrui Meng] fix binary metrics unit tests
db58a55 [Xiangrui Meng] add unit tests
a6e35d6 [Xiangrui Meng] minor update
60be09e [Xiangrui Meng] remove importance sampler
9ee9992 [Xiangrui Meng] update range partitioner to run only one job on roughly balanced data
cc12f47 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part
06ac2ec [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part
17bcbf3 [Reynold Xin] Added seed.
badf20d [Reynold Xin] Renamed the method.
6940010 [Reynold Xin] Reservoir sampling implementation.
|
|
|
|
|
|
|
|
| |
Author: Josh Rosen <joshrosen@apache.org>
Closes #1626 from JoshRosen/SPARK-2305 and squashes the following commits:
03fb283 [Josh Rosen] Update Py4J to version 0.8.2.1.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
sample size
Implemented stratified sampling that guarantees exact sample size using ScaRSR with two passes over the RDD for sampling without replacement and three passes for sampling with replacement.
Author: Doris Xin <doris.s.xin@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>
Closes #1025 from dorx/stratified and squashes the following commits:
245439e [Doris Xin] moved minSamplingRate to getUpperBound
eaf5771 [Doris Xin] bug fixes.
17a381b [Doris Xin] fixed a merge issue and a failed unit
ea7d27f [Doris Xin] merge master
b223529 [Xiangrui Meng] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java
b3013a4 [Xiangrui Meng] move math3 back to test scope
eecee5f [Doris Xin] Merge branch 'master' into stratified
f4c21f3 [Doris Xin] Reviewer comments
a10e68d [Doris Xin] style fix
a2bf756 [Doris Xin] Merge branch 'master' into stratified
680b677 [Doris Xin] use mapPartitionWithIndex instead
9884a9f [Doris Xin] style fix
bbfb8c9 [Doris Xin] Merge branch 'master' into stratified
ee9d260 [Doris Xin] addressed reviewer comments
6b5b10b [Doris Xin] Merge branch 'master' into stratified
254e03c [Doris Xin] minor fixes and Java API.
4ad516b [Doris Xin] remove unused imports from PairRDDFunctions
bd9dc6e [Doris Xin] unit bug and style violation fixed
1fe1cff [Doris Xin] Changed fractionByKey to a map to enable arg check
944a10c [Doris Xin] [SPARK-2145] Add lower bound on sampling rate
0214a76 [Doris Xin] cleanUp
90d94c0 [Doris Xin] merge master
9e74ab5 [Doris Xin] Separated out most of the logic in sampleByKey
7327611 [Doris Xin] merge master
50581fc [Doris Xin] added a TODO for logging in python
46f6c8c [Doris Xin] fixed the NPE caused by closures being cleaned before being passed into the aggregate function
7e1a481 [Doris Xin] changed the permission on SamplingUtil
1d413ce [Doris Xin] fixed checkstyle issues
9ee94ee [Doris Xin] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size
e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample
7cab53a [Doris Xin] fixed import bug in rdd.py
ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD
1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Datetime and time in Python will be converted into java.util.Calendar after serialization, it will be converted into java.sql.Timestamp during inferSchema().
In javaToPython(), Timestamp will be converted into Calendar, then be converted into datetime in Python after pickling.
Author: Davies Liu <davies.liu@gmail.com>
Closes #1601 from davies/date and squashes the following commits:
f0599b0 [Davies Liu] remove tests for sets and tuple in sql, fix list of list
c9d607a [Davies Liu] convert datetype for runtime
709d40d [Davies Liu] remove brackets
96db384 [Davies Liu] support datetime type for SchemaRDD
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The pull request includes two changes:
1. Removes SortOrder introduced by SPARK-2125. The key ordering already includes the SortOrder information since an Ordering can be reverse. This is similar to Java's Comparator interface. Rarely does an API accept both a Comparator as well as a SortOrder.
2. Replaces the sortWith call in HashShuffleReader with an in-place quick sort.
Author: Reynold Xin <rxin@apache.org>
Closes #1631 from rxin/sortOrder and squashes the following commits:
c9d37e1 [Reynold Xin] [SPARK-2726] and [SPARK-2727] Remove SortOrder and do in-place sort.
|