Before we would get the following (benign) error if we called `sc.stop()` twice. This is because the listener bus would try to post the end event again even after it has already stopped. This happens occasionally when flaky tests fail, usually as a result of other sources of error. Either way we shouldn't be logging this error when it is not the cause of the failure.
```
ERROR LiveListenerBus: SparkListenerBus has already stopped! Dropping event SparkListenerApplicationEnd(1425348445682)
```
Author: Andrew Or <andrew@databricks.com>
Closes#4871 from andrewor14/sc-stop and squashes the following commits:
a14afc5 [Andrew Or] Move code after code
915db16 [Andrew Or] Move code into code
The problem is that `ContextCleaner` may clean variables that belong to a different `SparkContext`. This can happen if the `SparkContext` to which the cleaner belongs stops, and a new one is started immediately afterwards in the same JVM. In this case, if the cleaner is in the middle of cleaning a broadcast, for instance, it will do so through `SparkEnv.get.blockManager`, which could be one that belongs to a different `SparkContext`.
JoshRosen and I suspect that this is the cause of many flaky tests, most notably the `JavaAPISuite`. We were able to reproduce the failure locally (though it is not deterministic and very hard to reproduce).
Author: Andrew Or <andrew@databricks.com>
Closes#4869 from andrewor14/cleaner-masquerade and squashes the following commits:
29168c0 [Andrew Or] Synchronize ContextCleaner stop
Add warning about building with Java 7+ and running the JAR on early Java 6.
CC andrewor14
Author: Sean Owen <sowen@cloudera.com>
Closes#4874 from srowen/SPARK-1911 and squashes the following commits:
79fa2f6 [Sean Owen] Add warning about building with Java 7+ and running the JAR on early Java 6.
Use array copy instead of `Iterator#toArray` to make it more efficient.
Author: Wenchen Fan <cloud0fan@outlook.com>
Closes#4825 from cloud-fan/minor and squashes the following commits:
c933ee5 [Wenchen Fan] make toArray method just in parent
946a35b [Wenchen Fan] minor enhance
https://issues.apache.org/jira/browse/SPARK-6118
I found that the object CommandUtils is placed under deploy.worker package, while CommandUtilsSuite is under deploy
Conventionally, we put the implementation and unit test class under the same package
here, to minimize the change, I move CommandUtilsSuite to worker package,
**However, CommandUtils seems to contain some general methods (though only used by worker.* classes currently**, we may also consider to replace CommonUtils
Author: CodingCat <zhunansjtu@gmail.com>
Closes#4856 from CodingCat/SPARK-6118 and squashes the following commits:
cb93700 [CodingCat] making package name consistent
This adds two features:
1. The ability to publish with a different maven version than
that specified in the release source.
2. Forking of different Zinc instances during the parallel dist
creation (to help with some stability issues).
Adding more description on top of #4861.
Author: DB Tsai <dbtsai@alpinenow.com>
Closes#4866 from dbtsai/doc and squashes the following commits:
37e9d07 [DB Tsai] doc
Issue: When the Python DecisionTree example in the programming guide is run, it runs out of Java Heap Space when using the default memory settings for the spark shell.
This prints a warning.
CC: mengxr
Author: Joseph K. Bradley <joseph@databricks.com>
Closes#4864 from jkbradley/dt-save-heap and squashes the following commits:
02e8daf [Joseph K. Bradley] fixed based on code review
7ecb1ed [Joseph K. Bradley] Added warnings about memory when calling tree and ensemble model save with too small a Java heap size
Similar to `MatrixFactorizaionModel`, we only need wrappers to support save/load for tree models in Python.
jkbradley
Author: Xiangrui Meng <meng@databricks.com>
Closes#4854 from mengxr/SPARK-6097 and squashes the following commits:
4586a4d [Xiangrui Meng] fix more typos
8ebcac2 [Xiangrui Meng] fix python style
91172d8 [Xiangrui Meng] fix typos
201b3b9 [Xiangrui Meng] update user guide
b5158e2 [Xiangrui Meng] support tree model save/load in PySpark/MLlib
- Various Fixes to docs
- Make data source traits actually interfaces
Based on #4862 but with fixed conflicts.
Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#4868 from marmbrus/pr/4862 and squashes the following commits:
fe091ea [Michael Armbrust] Merge remote-tracking branch 'origin/master' into pr/4862
0208497 [Reynold Xin] Test fixes.
34e0a28 [Reynold Xin] [SPARK-5310][SQL] Various fixes to Spark SQL docs.
This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However, the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability, `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.
JIRA: https://issues.apache.org/jira/browse/SPARK-5950
Thanks viirya for the initial work in #4729.
cc marmbrus liancheng
Author: Yin Huai <yhuai@databricks.com>
Closes#4826 from yhuai/insertNullabilityCheck and squashes the following commits:
3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.
davies
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#4860 from tdas/SPARK-6127 and squashes the following commits:
82de92a [Tathagata Das] Add Kafka to Python api docs
This is based on #4801 from dbtsai. The linear method guide is re-organized a little bit for this change.
Closes#4801
Author: Xiangrui Meng <meng@databricks.com>
Author: DB Tsai <dbtsai@alpinenow.com>
Closes#4861 from mengxr/SPARK-5537 and squashes the following commits:
47af0ac [Xiangrui Meng] update user guide for multinomial logistic regression
cdc2e15 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into AlpineNow-mlor-doc
096d0ca [DB Tsai] first commit
`df.dtypes` shows `null` for UDTs. This PR uses `udt` by default and `VectorUDT` overwrites it with `vector`.
jkbradley davies
Author: Xiangrui Meng <meng@databricks.com>
Closes#4858 from mengxr/SPARK-6121 and squashes the following commits:
34f0a77 [Xiangrui Meng] simpleString for UDT
Some memory not count into memory used by memoryStore or unrollMemory.
Thread A after unrollsafely memory, it will release 40MB unrollMemory(40MB will used by other threads). then ThreadA wait get accountingLock to tryToPut blockA(30MB). before Thread A get accountingLock, blockA memory size is not counting into unrollMemory or memoryStore.currentMemory.
IIUC, freeMemory should minus that block memory
So, put this release memory into pending, and release it in tryToPut before ensureSpace
Author: hushan[胡珊] <hushan@xiaomi.com>
Closes#3629 from suyanNone/unroll-memory and squashes the following commits:
809cc41 [hushan[胡珊]] Refine
407b2c9 [hushan[胡珊]] Refine according comments
39960d0 [hushan[胡珊]] Refine comments
0fd0213 [hushan[胡珊]] add comments
0fc2bec [hushan[胡珊]] Release pending unroll memory after put block in memoryStore
3a3f2c8 [hushan[胡珊]] Refine blockManagerSuite unroll test
3323c45 [hushan[胡珊]] Refine getOrElse
f664317 [hushan[胡珊]] Make sure not add pending in every releaseUnrollMemory call
08b32ba [hushan[胡珊]] Pending unroll memory for this block untill tryToPut
There are multiple issues with translating on set outlined in the JIRA.
This PR reverts the translation logic added to `SparkConf`. In the future, after the 1.3.0 release we will figure out a way to reorganize the internal structure more elegantly. For now, let's preserve the existing semantics of `SparkConf` since it's a public interface. Unfortunately this means duplicating some code for now, but this is all internal and we can always clean it up later.
Author: Andrew Or <andrew@databricks.com>
Closes#4799 from andrewor14/conf-set-translate and squashes the following commits:
11c525b [Andrew Or] Move warning to driver
10e77b5 [Andrew Or] Add documentation for deprecation precedence
a369cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into conf-set-translate
c26a9e3 [Andrew Or] Revert all translate logic in SparkConf
fef6c9c [Andrew Or] Restore deprecation logic for spark.executor.userClassPathFirst
94b4dfa [Andrew Or] Translate on get, not set
Some users have reported difficulty in parsing the new event log format. Since we embed the metadata in the beginning of the file, when we compress the event log we need to skip the metadata because we need that information to parse the log later. This means we'll end up with a partially compressed file if event logging compression is turned on. The old format looks like:
```
sparkVersion = 1.3.0
compressionCodec = org.apache.spark.io.LZFCompressionCodec
=== LOG_HEADER_END ===
// actual events, could be compressed bytes
```
The new format in this patch puts the compression codec in the log file name instead. It also removes the metadata header altogether along with the Spark version, which was not needed. The new file name looks something like:
```
app_without_compression
app_123.lzf
app_456.snappy
```
I tested this with and without compression, using different compression codecs and event logging directories. I verified that both the `Master` and the `HistoryServer` can render both compressed and uncompressed logs as before.
Author: Andrew Or <andrew@databricks.com>
Closes#4821 from andrewor14/event-log-format and squashes the following commits:
8511141 [Andrew Or] Fix test
654883d [Andrew Or] Add back metadata with Spark version
7f537cd [Andrew Or] Address review feedback
7d6aa61 [Andrew Or] Make codec an extension
59abee9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
27c9a6c [Andrew Or] Address review feedback
519e51a [Andrew Or] Address review feedback
ef69276 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
88a091d [Andrew Or] Add tests for new format and file name
f32d8d2 [Andrew Or] Fix tests
8db5a06 [Andrew Or] Embed metadata in the event log file name instead
Constructs like Hive `TRANSFORM` may generate malformed rows (via badly authored external scripts for example). I'm a bit hesitant to have this feature, since it introduces per-tuple cost when caching tables. However, considering caching tables is usually a one-time cost, this is probably worth having.
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4842)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4842 from liancheng/spark-6082 and squashes the following commits:
b05dbff [Cheng Lian] Provides better error message for malformed rows when caching tables
Author: Michael Armbrust <michael@databricks.com>
Closes#4855 from marmbrus/explodeBug and squashes the following commits:
a712249 [Michael Armbrust] [SPARK-6114][SQL] Avoid metastore conversions before plan is resolved
When starting the history server, all the log files will be fetched and parsed in order to get the applications' meta data e.g. App Name, Start Time, Duration, etc. In our production cluster, there exist 2600 log files (160G) in HDFS and it costs 3 hours to restart the history server, which is a little bit too long for us.
It would be better, if the history server can show logs with missing information during start-up and fill the missing information after fetching and parsing a log file.
Author: guliangliang <guliangliang@qiyi.com>
Closes#4525 from marsishandsome/Spark5522 and squashes the following commits:
a865c11 [guliangliang] fix bug2
4340c2b [guliangliang] fix bug
af92a5a [guliangliang] [SPARK-5522] Accelerate the Histroty Server start
Some YARN configurations return a vcore count for allocated
containers that does not match the requested resource. That means
Spark would always ignore those containers. So relax the the matching
of the vcore count to allow the Spark jobs to run.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#4818 from vanzin/SPARK-6050 and squashes the following commits:
991c803 [Marcelo Vanzin] Remove config option, standardize on legacy behavior (no vcore matching).
8c9c346 [Marcelo Vanzin] Restrict lax matching to vcores only.
3359692 [Marcelo Vanzin] [SPARK-6050] [yarn] Add config option to do lax resource matching.
HiveQL expression like `select count(1) from src tablesample(1 percent);` means take 1% sample to select. But it means 100% in the current version of the Spark.
Author: q00251598 <qiyadong@huawei.com>
Closes#4789 from watermen/SPARK-6040 and squashes the following commits:
2453ebe [q00251598] check and adjust the fraction.
It should be `true` instead of `false`?
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#4762 from viirya/doc_fix and squashes the following commits:
2e37482 [Liang-Chi Hsieh] Fix doc.
Point "Community" to main Spark Community page; mention SO tag apache-spark.
Separately, the Apache site can be updated to mention, under Mailing Lists:
"StackOverflow also has an apache-spark tag for Spark Q&A." or similar.
Author: Sean Owen <sowen@cloudera.com>
Closes#4843 from srowen/SPARK-5390 and squashes the following commits:
3508ac6 [Sean Owen] Point "Community" to main Spark Community page; mention SO tag apache-spark
The API signatire for join requires the JoinType to be the third parameter. The code examples provided for join show JoinType being provided as the 2nd parater resuling in errors (i.e. "df1.join(df2, "outer", $"df1Key" === $"df2Key") ). The correct sample code is df1.join(df2, $"df1Key" === $"df2Key", "outer")
Author: Paul Power <paul.power@peerside.com>
Closes#4847 from peerside/master and squashes the following commits:
ebc1efa [Paul Power] Merge pull request #1 from peerside/peerside-patch-1
e353340 [Paul Power] Updated comments use correct sample code for Dataframe joins
Currently LogisticRegressionWithLBFGS in python/pyspark/mllib/classification.py will invoke callMLlibFunc with a wrong "regType" parameter.
It was assigned to "str(regType)" which translate None(Python) to "None"(Java/Scala). The right way should be translate None(Python) to null(Java/Scala) just as what we did at LogisticRegressionWithSGD.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#4831 from yanboliang/pyspark_classification and squashes the following commits:
12db65a [Yanbo Liang] correct LogisticRegressionWithLBFGS regType parameter for pyspark
Examples illustrating difference between legacy mapReduceTriplets usage and aggregateMessages usage has type issues on the reduce for both operators.
Being just an example- changed example to reduce the message String by concatenation. Although non-optimal for performance.
Author: DEBORAH SIEGEL <deborahsiegel@DEBORAHs-MacBook-Pro.local>
Closes#4853 from d3borah/master and squashes the following commits:
db54173 [DEBORAH SIEGEL] fixed aggregateMessages example in graphX doc
When run ```select * from nzhang_part where hr = 'file,';```, it throws exception ```java.lang.IllegalArgumentException: Can not create a Path from an empty string```
. Because the path of hdfs contains comma, and FileInputFormat.setInputPaths will split path by comma.
### SQL
```
set hive.merge.mapfiles=true;
set hive.merge.mapredfiles=true;
set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
set hive.exec.dynamic.partition=true;
set hive.exec.dynamic.partition.mode=nonstrict;
create table nzhang_part like srcpart;
insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08';
insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';
insert overwrite table nzhang_part partition (ds='2010-08-15', hr)
select * from (
select key, value, hr from srcpart where ds='2008-04-08'
union all
select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
select * from nzhang_part where hr = 'file,';
```
### Error Log
```
15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part where hr = 'file,']
java.lang.IllegalArgumentException: Can not create a Path from an empty string
at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
at org.apache.hadoop.fs.Path.<init>(Path.java:135)
at org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at scala.Option.map(Option.scala:145)
at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
Author: q00251598 <qiyadong@huawei.com>
Closes#4532 from watermen/SPARK-5741 and squashes the following commits:
9758ab1 [q00251598] fix bug
1db1a1c [q00251598] use setInputPaths(Job job, Path... inputPaths)
b788a72 [q00251598] change FileInputFormat.setInputPaths to jobConf.set and add test suite
Usage info in documentation does not match actual usage info.
Doc string usage says ```Usage: network_wordcount.py <zk> <topic>``` whereas the actual usage is ```Usage: kafka_wordcount.py <zk> <topic>```
Author: Kenneth Myers <myerske@us.ibm.com>
Closes#4852 from kennethmyers/kafka_wordcount_documentation_fix and squashes the following commits:
3855325 [Kenneth Myers] Fixed usage string in documentation.
Always set `containsNull = true` when infer the schema of JSON datasets. If we set `containsNull` based on records we scanned, we may miss arrays with null values when we do sampling. Also, because future data can have arrays with null values, if we convert JSON data to parquet, always setting `containsNull = true` is a more robust way to go.
JIRA: https://issues.apache.org/jira/browse/SPARK-6052
Author: Yin Huai <yhuai@databricks.com>
Closes#4806 from yhuai/jsonArrayContainsNull and squashes the following commits:
05eab9d [Yin Huai] Change containsNull to true.
JIRA: https://issues.apache.org/jira/browse/SPARK-6073
liancheng
Author: Yin Huai <yhuai@databricks.com>
Closes#4824 from yhuai/refreshCache and squashes the following commits:
b9542ef [Yin Huai] Refresh metadata cache in the Catalog in CreateMetastoreDataSourceAsSelect.
Class TaskContext is unused in EdgeRDDImpl, so we need to remove it from import list.
Author: Lianhui Wang <lianhuiwang09@gmail.com>
Closes#4846 from lianhuiwang/SPARK-6103 and squashes the following commits:
31aed64 [Lianhui Wang] remove unused class to import in EdgeRDDImpl
Demote some 'noisy' log messages to debug level. I added a few more, to include everything that gets logged in stanzas like this:
```
15/03/01 00:03:54 INFO BlockManager: Removing broadcast 0
15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0_piece0
15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0_piece0 of size 839 dropped from memory (free 277976091)
15/03/01 00:03:54 INFO BlockManagerInfo: Removed broadcast_0_piece0 on localhost:49524 in memory (size: 839.0 B, free: 265.1 MB)
15/03/01 00:03:54 INFO BlockManagerMaster: Updated info of block broadcast_0_piece0
15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0
15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0 of size 1088 dropped from memory (free 277977179)
15/03/01 00:03:54 INFO ContextCleaner: Cleaned broadcast 0
```
as well as regular messages like
```
15/03/01 00:02:33 INFO MemoryStore: ensureFreeSpace(2640) called with curMem=47322, maxMem=278019440
```
WDYT? good or should some be left alone?
CC mengxr who suggested some of this.
Author: Sean Owen <sowen@cloudera.com>
Closes#4838 from srowen/SPARK-3357 and squashes the following commits:
dce75c1 [Sean Owen] Back out some debug level changes
d9b784d [Sean Owen] Demote some 'noisy' log messages to debug level
Small changes, please help to review, thanks a lot.
Author: Saisai Shao <saisai.shao@intel.com>
Closes#4837 from jerryshao/doc-fix and squashes the following commits:
545291a [Saisai Shao] Fix some error docs in streaming examples
Author: MechCoder <manojkumarsivaraj334@gmail.com>
Closes#4834 from MechCoder/spark-6083 and squashes the following commits:
1cdd7b5 [MechCoder] Add parse function
65bbbe9 [MechCoder] [SPARK-6083] Make Python API example consistent in NaiveBayes
A simple wrapper to save/load `MatrixFactorizationModel` in Python. jkbradley
Author: Xiangrui Meng <meng@databricks.com>
Closes#4811 from mengxr/SPARK-5991 and squashes the following commits:
f135dac [Xiangrui Meng] update save doc
57e5200 [Xiangrui Meng] address comments
06140a4 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5991
282ec8d [Xiangrui Meng] support save/load in PySpark's ALS
This is needed for the SQL bindings to work on Yarn.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#4822 from vanzin/SPARK-6074 and squashes the following commits:
fb52001 [Marcelo Vanzin] [SPARK-6074] [sql] Package pyspark sql bindings.
This fixes a non-deterministic bug introduced in #4021 that could cause tasks' accumulator updates to be lost. The problem is that `localAccums` should not hold weak references: after the task finishes running there won't be any strong references to these local accumulators, so they can get garbage-collected before the executor reads the `localAccums` map. We don't need weak references here anyways, since this map is cleared at the end of each task.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#4835 from JoshRosen/SPARK-6075 and squashes the following commits:
4f4b5b2 [Josh Rosen] Remove defensive assertions that caused test failures in code unrelated to this change
120c7b0 [Josh Rosen] [SPARK-6075] Do not store WeakReferences in localAccums map
Don't throw NPE if appId is unknown. kayousterhout is this a decent enough band-aid for avoiding a full-blown NPE? it should just render empty content instead
Author: Sean Owen <sowen@cloudera.com>
Closes#4777 from srowen/SPARK-1965 and squashes the following commits:
7e16590 [Sean Owen] Update app not found message
cb878d6 [Sean Owen] Return basic "not found" page for unknown appId
d8270da [Sean Owen] Don't throw NPE if appId is unknown
Disallow TRACE HTTP method in servlets
Author: Sean Owen <sowen@cloudera.com>
Closes#4765 from srowen/SPARK-5983 and squashes the following commits:
421b25b [Sean Owen] Disallow TRACE HTTP method in servlets
Remove unicode characters from MLlib file.
Author: Michael Griffiths <msjgriffiths@gmail.com>
Author: Griffiths, Michael (NYC-RPM) <michael.griffiths@reprisemedia.com>
Closes#4815 from msjgriffiths/SPARK-6063 and squashes the following commits:
bcd7de1 [Griffiths, Michael (NYC-RPM)] Change \u201D quote marks around 'theta' to standard single apostrophe (\x27)
38eb535 [Michael Griffiths] Merge pull request #2 from apache/master
b08e865 [Michael Griffiths] Merge pull request #1 from apache/master
This commit exists to close the following pull requests on Github:
Closes#1128 (close requested by 'srowen')
Closes#3425 (close requested by 'srowen')
Closes#4770 (close requested by 'srowen')
Closes#2813 (close requested by 'srowen')
pwendell tdas
This is the safer parts of PR #4754:
- SPARK-5979: All dependencies with the groupId `org.apache.spark` passed through `--packages`, were being excluded from the dependency tree on the assumption that they would be in the assembly jar. This is not the case, therefore the exclusion rules had to be defined more explicitly.
- SPARK-6032: Ivy prints a whole lot of logs while retrieving dependencies. These were printed to `System.out`. Moved the logging to `System.err`.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#4802 from brkyvz/simple-streaming-fix and squashes the following commits:
e0f38cb [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into simple-streaming-fix
bad921c [Burak Yavuz] [SPARK-5979][SPARK-6032] Smaller safer fix
These may conflict with the classes already in the NM. We shouldn't
be repackaging them.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#4820 from vanzin/SPARK-6070 and squashes the following commits:
871b566 [Marcelo Vanzin] The "d'oh how didn't I think of it before" solution.
3cba946 [Marcelo Vanzin] Use profile instead, so that dependencies don't need to be explicitly listed.
7a18a1b [Marcelo Vanzin] [SPARK-6070] [yarn] Remove unneeded classes from shuffle service jar.
The _eq_ of DataType is not correct, class cache is not use correctly (created class can not be find by dataType), then it will create lots of classes (saved in _cached_cls), never released.
Also, all same DataType have same hash code, there will be many object in a dict with the same hash code, end with hash attach, it's very slow to access this dict (depends on the implementation of CPython).
This PR also improve the performance of inferSchema (avoid the unnecessary converter of object).
cc pwendell JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#4808 from davies/leak and squashes the following commits:
6a322a4 [Davies Liu] tests refactor
3da44fc [Davies Liu] fix __eq__ of Singleton
534ac90 [Davies Liu] add more checks
46999dc [Davies Liu] fix tests
d9ae973 [Davies Liu] fix memory leak in sql
This is a follow-up of #4720. By default, `spark-daemon.sh` writes PID files under `/tmp`, which makes it impossible to start multiple server instances simultaneously. This PR sets `SPARK_PID_DIR` to Spark home directory to workaround this problem.
Many thanks to chenghao-intel for pointing out this issue!
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4758)
<!-- Reviewable:end -->
Author: Cheng Lian <lian@databricks.com>
Closes#4758 from liancheng/thriftserver-pid-dir and squashes the following commits:
252fa0f [Cheng Lian] Uses temporary directory as Thrift server PID directory
1b3d1e3 [Cheng Lian] Sets SPARK_HOME as SPARK_PID_DIR when running Thrift server test suites