aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* SPARK-6085 Increase default value for memory overheadtedyu2015-03-044-5/+5
| | | | | | | | | Author: tedyu <yuzhihong@gmail.com> Closes #4836 from tedyu/master and squashes the following commits: d65b495 [tedyu] SPARK-6085 Increase default value for memory overhead 1fdd4df [tedyu] SPARK-6085 Increase default value for memory overhead
* [SPARK-6141][MLlib] Upgrade Breeze from 0.10 to 0.11 to fix convergence bugXiangrui Meng2015-03-032-1/+5
| | | | | | | | | | | | | | | | | | LBFGS and OWLQN in Breeze 0.10 has convergence check bug. This is fixed in 0.11, see the description in Breeze project for detail: https://github.com/scalanlp/breeze/pull/373#issuecomment-76879760 Author: Xiangrui Meng <meng@databricks.com> Author: DB Tsai <dbtsai@alpinenow.com> Author: DB Tsai <dbtsai@dbtsai.com> Closes #4879 from dbtsai/breeze and squashes the following commits: d848f65 [DB Tsai] Merge pull request #1 from mengxr/AlpineNow-breeze c2ca6ac [Xiangrui Meng] upgrade to breeze-0.11.1 35c2f26 [Xiangrui Meng] fix LRSuite 397a208 [DB Tsai] upgrade breeze
* [SPARK-6132][HOTFIX] ContextCleaner InterruptedException should be quietAndrew Or2015-03-031-0/+1
| | | | | | | | | | If the cleaner is stopped, we shouldn't print a huge stack trace when the cleaner thread is interrupted because we purposefully did this. Author: Andrew Or <andrew@databricks.com> Closes #4882 from andrewor14/cleaner-interrupt and squashes the following commits: 8652120 [Andrew Or] Just a hot fix
* [SPARK-5949] HighlyCompressedMapStatus needs more classes registered w/ kryoImran Rashid2015-03-032-5/+33
| | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-5949 Author: Imran Rashid <irashid@cloudera.com> Closes #4877 from squito/SPARK-5949_register_roaring_bitmap and squashes the following commits: 7e13316 [Imran Rashid] style style style 5f6bb6d [Imran Rashid] more style 709bfe0 [Imran Rashid] style a5cb744 [Imran Rashid] update tests to cover both types of RoaringBitmapContainers 09610c6 [Imran Rashid] formatting f9a0b7c [Imran Rashid] put primitive array registrations together 97beaf8 [Imran Rashid] SPARK-5949 HighlyCompressedMapStatus needs more classes registered w/ kryo
* [SPARK-6133] Make sc.stop() idempotentAndrew Or2015-03-031-2/+2
| | | | | | | | | | | | | | 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
* [SPARK-6132] ContextCleaner race condition across SparkContextsAndrew Or2015-03-031-13/+26
| | | | | | | | | | | | 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
* SPARK-1911 [DOCS] Warn users if their assembly jars are not built with Java 6Sean Owen2015-03-031-0/+4
| | | | | | | | | | | | 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.
* Revert "[SPARK-5423][Core] Cleanup resources in DiskMapIterator.finalize to ↵Andrew Or2015-03-031-43/+9
| | | | | | ensure deleting the temp file" This reverts commit 90095bf3ce9304d09a32ceffaa99069079071b59.
* [SPARK-6138][CORE][minor] enhance the `toArray` method in `SizeTrackingVector`Wenchen Fan2015-03-032-10/+12
| | | | | | | | | | | 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
* [SPARK-6118] making package name of deploy.worker.CommandUtils and ↵CodingCat2015-03-031-3/+2
| | | | | | | | | | | | | | | | | | | | deploy.CommandUtilsSuite consistent 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
* BUILD: Minor tweaks to internal build scriptsPatrick Wendell2015-03-031-5/+19
| | | | | | | | 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).
* HOTFIX: Bump HBase version in MapR profiles.Patrick Wendell2015-03-031-2/+2
| | | | After #2982 (SPARK-4048) we rely on the newer HBase packaging format.
* [SPARK-5537][MLlib][Docs] Add user guide for multinomial logistic regressionDB Tsai2015-03-021-0/+10
| | | | | | | | | | 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
* [SPARK-6120] [mllib] Warnings about memory in tree, ensemble model saveJoseph K. Bradley2015-03-022-4/+50
| | | | | | | | | | | | | | | 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
* [SPARK-6097][MLLIB] Support tree model save/load in PySpark/MLlibXiangrui Meng2015-03-026-33/+109
| | | | | | | | | | | | | | | | 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
* [SPARK-5310][SQL] Fixes to Docs and Datasources APIReynold Xin2015-03-0222-136/+115
| | | | | | | | | | | | | | | | - 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.
* [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should ↵Yin Huai2015-03-0217-36/+330
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | work when using datasource api 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.
* [SPARK-6127][Streaming][Docs] Add Kafka to Python api docsTathagata Das2015-03-021-0/+7
| | | | | | | | | | 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
* [SPARK-5537] Add user guide for multinomial logistic regressionXiangrui Meng2015-03-021-61/+217
| | | | | | | | | | | | | | | 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
* [SPARK-6121][SQL][MLLIB] simpleString for UDTXiangrui Meng2015-03-022-1/+4
| | | | | | | | | | | | `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
* [SPARK-4777][CORE] Some block memory after unrollSafely not count into used ↵hushan[胡珊]2015-03-022-7/+47
| | | | | | | | | | | | | | | | | | | | | | | | memory(memoryStore.entrys or unrollMemory) 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
* [SPARK-6048] SparkConf should not translate deprecated configs on setAndrew Or2015-03-025-22/+25
| | | | | | | | | | | | | | | | | 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
* [SPARK-6066] Make event log format easier to parseAndrew Or2015-03-0214-189/+212
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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
* [SPARK-6082] [SQL] Provides better error message for malformed rows when ↵Cheng Lian2015-03-021-0/+11
| | | | | | | | | | | | | | | | caching tables 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
* [SPARK-6114][SQL] Avoid metastore conversions before plan is resolvedMichael Armbrust2015-03-022-0/+14
| | | | | | | | 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
* [SPARK-5522] Accelerate the Histroty Server startguliangliang2015-03-021-41/+74
| | | | | | | | | | | | | | 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
* [SPARK-6050] [yarn] Relax matching of vcore count in received containers.Marcelo Vanzin2015-03-021-2/+8
| | | | | | | | | | | | | | | 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.
* [SPARK-6040][SQL] Fix the percent bug in tablesampleq002515982015-03-022-1/+11
| | | | | | | | | | 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.
* [Minor] Fix doc typo for describing primitiveTerm effectiveness conditionLiang-Chi Hsieh2015-03-021-1/+1
| | | | | | | | | | 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.
* SPARK-5390 [DOCS] Encourage users to post on Stack Overflow in Community DocsSean Owen2015-03-021-8/+2
| | | | | | | | | | | | | 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
* [DOCS] Refactored Dataframe join comment to use correct parameter orderingPaul Power2015-03-021-2/+2
| | | | | | | | | | | 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
* [SPARK-6080] [PySpark] correct LogisticRegressionWithLBFGS regType parameter ↵Yanbo Liang2015-03-021-1/+1
| | | | | | | | | | | | | for pyspark 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
* aggregateMessages example in graphX docDEBORAH SIEGEL2015-03-021-2/+2
| | | | | | | | | | | | 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
* [SPARK-5741][SQL] Support the path contains comma in HiveContextq002515982015-03-0218-1/+2511
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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
* [SPARK-6111] Fixed usage string in documentation.Kenneth Myers2015-03-021-1/+1
| | | | | | | | | | | | 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.
* [SPARK-6052][SQL]In JSON schema inference, we should always set containsNull ↵Yin Huai2015-03-022-24/+23
| | | | | | | | | | | | | | of an ArrayType to true 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.
* [SPARK-6073][SQL] Need to refresh metastore cache after append data in ↵Yin Huai2015-03-022-0/+54
| | | | | | | | | | | | | | CreateMetastoreDataSourceAsSelect 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.
* [SPARK-6103][Graphx]remove unused class to import in EdgeRDDImplLianhui Wang2015-03-021-1/+1
| | | | | | | | | | 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
* SPARK-3357 [CORE] Internal log messages should be set at DEBUG level instead ↵Sean Owen2015-03-024-6/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | of INFO 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
* [Streaming][Minor]Fix some error docs in streaming examplesSaisai Shao2015-03-023-3/+4
| | | | | | | | | | 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
* [SPARK-6083] [MLLib] [DOC] Make Python API example consistent in NaiveBayesMechCoder2015-03-011-10/+16
| | | | | | | | | 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
* [SPARK-6053][MLLIB] support save/load in PySpark's ALSXiangrui Meng2015-03-014-6/+82
| | | | | | | | | | | | | 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
* [SPARK-6074] [sql] Package pyspark sql bindings.Marcelo Vanzin2015-03-011-0/+8
| | | | | | | | | | 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.
* [SPARK-6075] Fix bug in that caused lost accumulator updates: do not store ↵Josh Rosen2015-02-281-17/+23
| | | | | | | | | | | | | WeakReferences in localAccums map 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
* SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsExceptionEvan Yu2015-02-284-5/+161
| | | | | | | | | | | | | | | | | | Fix TimSort bug which causes a ArrayOutOfBoundsException. Using the proposed fix here http://envisage-project.eu/proving-android-java-and-python-sorting-algorithm-is-broken-and-how-to-fix-it/ Author: Evan Yu <ehotou@gmail.com> Closes #4804 from hotou/SPARK-5984 and squashes the following commits: 3421b6c [Evan Yu] SPARK-5984: Add info to LICENSE e61c6b8 [Evan Yu] SPARK-5984: Fix license and document 6ccc280 [Evan Yu] SPARK-5984: Add License header to file e06c0d2 [Evan Yu] SPARK-5984: Add License header to file 4d95f75 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException 479a106 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException
* SPARK-1965 [WEBUI] Spark UI throws NPE on trying to load the app page for ↵Sean Owen2015-02-281-1/+10
| | | | | | | | | | | | | | non-existent app 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
* SPARK-5983 [WEBUI] Don't respond to HTTP TRACE in HTTP-based UIsSean Owen2015-02-282-0/+12
| | | | | | | | | | 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
* SPARK-6063 MLlib doesn't pass mvn scalastyle check due to UTF chars in ↵Michael Griffiths2015-02-281-1/+1
| | | | | | | | | | | | | | | LDAModel.scala 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
* [SPARK-5775] [SQL] BugFix: GenericRow cannot be cast to SpecificMutableRow ↵Cheng Lian2015-02-283-24/+217
| | | | | | | | | | | | | | | | | | | | | when nested data and partitioned table This PR adapts anselmevignon's #4697 to master and branch-1.3. Please refer to PR description of #4697 for details. <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4792) <!-- Reviewable:end --> Author: Cheng Lian <lian@databricks.com> Author: Cheng Lian <liancheng@users.noreply.github.com> Author: Yin Huai <yhuai@databricks.com> Closes #4792 from liancheng/spark-5775 and squashes the following commits: 538f506 [Cheng Lian] Addresses comments cee55cf [Cheng Lian] Merge pull request #4 from yhuai/spark-5775-yin b0b74fb [Yin Huai] Remove runtime pattern matching. ca6e038 [Cheng Lian] Fixes SPARK-5775
* MAINTENANCE: Automated closing of pull requests.Patrick Wendell2015-02-270-0/+0
| | | | | | | | | 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')