aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* replace awaitTransformation with awaitTermination in scaladoc/javadocHolden Karau2014-10-213-3/+3
| | | | | | | | | Author: Holden Karau <holden@pigscanfly.ca> Closes #2861 from holdenk/SPARK-4015-Documentation-in-the-streaming-context-references-non-existent-function and squashes the following commits: 081db8a [Holden Karau] fix pyspark streaming doc too 0e03863 [Holden Karau] replace awaitTransformation with awaitTermination
* [SPARK-4023] [MLlib] [PySpark] convert rdd into RDD of VectorDavies Liu2014-10-212-4/+24
| | | | | | | | | | | | | Convert the input rdd to RDD of Vector. cc mengxr Author: Davies Liu <davies@databricks.com> Closes #2870 from davies/fix4023 and squashes the following commits: 1eac767 [Davies Liu] address comments 0871576 [Davies Liu] convert rdd into RDD of Vector
* [SPARK-3958] TorrentBroadcast cleanup / debugging improvements.Josh Rosen2014-10-213-92/+97
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR makes several changes to TorrentBroadcast in order to make it easier to reason about, which should help when debugging SPARK-3958. The key changes: - Remove all state from the global TorrentBroadcast object. This state consisted mainly of configuration options, like the block size and compression codec, and was read by the blockify / unblockify methods. Unfortunately, the use of `lazy val` for `BLOCK_SIZE` meant that the block size was always determined by the first SparkConf that TorrentBroadast was initialized with; as a result, unit tests could not properly test TorrentBroadcast with different block sizes. Instead, blockifyObject and unBlockifyObject now accept compression codecs and blockSizes as arguments. These arguments are supplied at the call sites inside of TorrentBroadcast instances. Each TorrentBroadcast instance determines these values from SparkEnv's SparkConf. I was careful to ensure that we do not accidentally serialize CompressionCodec or SparkConf objects as part of the TorrentBroadcast object. - Remove special-case handling of local-mode in TorrentBroadcast. I don't think that broadcast implementations should know about whether we're running in local mode. If we want to optimize the performance of broadcast in local mode, then we should detect this at a higher level and use a dummy LocalBroadcastFactory implementation instead. Removing this code fixes a subtle error condition: in the old local mode code, a failure to find the broadcast in the local BlockManager would lead to an attempt to deblockify zero blocks, which could lead to confusing deserialization or decompression errors when we attempted to decompress an empty byte array. This should never have happened, though: a failure to find the block in local mode is evidence of some other error. The changes here will make it easier to debug those errors if they ever happen. - Add a check that throws an exception when attempting to deblockify an empty array. - Use ScalaCheck to add a test to check that TorrentBroadcast's blockifyObject and unBlockifyObject methods are inverses. - Misc. cleanup and logging improvements. Author: Josh Rosen <joshrosen@databricks.com> Closes #2844 from JoshRosen/torrentbroadcast-bugfix and squashes the following commits: 1e8268d [Josh Rosen] Address Reynold's review comments 2a9fdfd [Josh Rosen] Address Reynold's review comments. c3b08f9 [Josh Rosen] Update TorrentBroadcast tests to reflect removal of special local-mode optimizations. 5c22782 [Josh Rosen] Store broadcast variable's value in the driver. 33fc754 [Josh Rosen] Change blockify/unblockifyObject to accept serializer as argument. 618a872 [Josh Rosen] [SPARK-3958] TorrentBroadcast cleanup / debugging improvements.
* Update Building Spark link.Reynold Xin2014-10-201-1/+1
|
* [SPARK-3940][SQL] Avoid console printing error messages three timeswangxiaojing2014-10-201-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | If wrong sql,the console print error one times。 eg: <pre> spark-sql> show tabless; show tabless; 14/10/13 21:03:48 INFO ParseDriver: Parsing command: show tabless ............ at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:274) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:209) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) Caused by: org.apache.hadoop.hive.ql.parse.ParseException: line 1:5 cannot recognize input near 'show' 'tabless' '<EOF>' in ddl statement at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:193) at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:161) at org.apache.spark.sql.hive.HiveQl$.getAst(HiveQl.scala:218) at org.apache.spark.sql.hive.HiveQl$.createPlan(HiveQl.scala:226) ... 47 more Time taken: 4.35 seconds 14/10/13 21:03:51 INFO CliDriver: Time taken: 4.35 seconds </pre> Author: wangxiaojing <u9jing@gmail.com> Closes #2790 from wangxiaojing/spark-3940 and squashes the following commits: e2e5c14 [wangxiaojing] sql Print the error code three times
* [SPARK-3969][SQL] Optimizer should have a super class as an interface.Takuya UESHIN2014-10-203-5/+7
| | | | | | | | | | | | Some developers want to replace `Optimizer` to fit their projects but can't do so because currently `Optimizer` is an `object`. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #2825 from ueshin/issues/SPARK-3969 and squashes the following commits: abbc53c [Takuya UESHIN] Re-rename Optimizer object. 4d2e1bc [Takuya UESHIN] Rename Optimizer object. 9547a23 [Takuya UESHIN] Extract abstract class from Optimizer for developers to be able to replace Optimizer.
* [SPARK-3945]Properties of hive-site.xml is invalid in running the Thrift ↵luogankun2014-10-201-1/+5
| | | | | | | | | | | | | | JDBC server Write properties of hive-site.xml to HiveContext when initilize session state in SparkSQLEnv.scala. The method of SparkSQLEnv.init() in HiveThriftServer2.scala can not write the properties of hive-site.xml to HiveContext. Such as: add configuration property spark.sql.shuffle.partititions in the hive-site.xml. Author: luogankun <luogankun@gmail.com> Closes #2800 from luogankun/SPARK-3945 and squashes the following commits: 3679efc [luogankun] [SPARK-3945]Write properties of hive-site.xml to HiveContext when initilize session state In SparkSQLEnv.scala
* [SPARK-3966][SQL] Fix nullabilities of Cast related to DateType.Takuya UESHIN2014-10-201-0/+4
| | | | | | | | Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #2820 from ueshin/issues/SPARK-3966 and squashes the following commits: ca4a745 [Takuya UESHIN] Fix nullabilities of Cast related to DateType.
* [SPARK-3800][SQL] Clean aliases from grouping expressionsMichael Armbrust2014-10-202-0/+35
| | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #2658 from marmbrus/nestedAggs and squashes the following commits: 862b763 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 3234521 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 8b06fdc [Michael Armbrust] possible fix for grouping on nested fields
* [SPARK-3906][SQL] Adds multiple join support for SQLContextCheng Lian2014-10-202-3/+16
| | | | | | | | Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #2767 from liancheng/multi-join and squashes the following commits: 9dc0d18 [Cheng Lian] Adds multiple join support for SQLContext
* [SPARK-3207][MLLIB]Choose splits for continuous features in DecisionTree ↵Qiping Li2014-10-205-16/+176
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | more adaptively DecisionTree splits on continuous features by choosing an array of values from a subsample of the data. Currently, it does not check for identical values in the subsample, so it could end up having multiple copies of the same split. In this PR, we choose splits for a continuous feature in 3 steps: 1. Sort sample values for this feature 2. Get number of occurrence of each distinct value 3. Iterate the value count array computed in step 2 to choose splits. After find splits, `numSplits` and `numBins` in metadata will be updated. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: Qiping Li <liqiping1991@gmail.com> Author: chouqin <liqiping1991@gmail.com> Author: liqi <liqiping1991@gmail.com> Author: qiping.lqp <qiping.lqp@alibaba-inc.com> Closes #2780 from chouqin/dt-findsplits and squashes the following commits: 18d0301 [Qiping Li] check explicitly findsplits return distinct splits 8dc28ab [chouqin] remove blank lines ffc920f [chouqin] adjust code based on comments and add more test cases 9857039 [chouqin] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits d353596 [qiping.lqp] fix pyspark doc test 9e64699 [Qiping Li] fix random forest unit test 3c72913 [Qiping Li] fix random forest unit test 092efcb [Qiping Li] fix bug f69f47f [Qiping Li] fix bug ab303a4 [Qiping Li] fix bug af6dc97 [Qiping Li] fix bug 2a8267a [Qiping Li] fix bug c339a61 [Qiping Li] fix bug 369f812 [Qiping Li] fix style 8f46af6 [Qiping Li] add comments and unit test 9e7138e [Qiping Li] Merge branch 'dt-findsplits' of https://github.com/chouqin/spark into dt-findsplits 1b25a35 [Qiping Li] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits 0cd744a [liqi] fix bug 3652823 [Qiping Li] fix bug af7cb79 [Qiping Li] Choose splits for continuous features in DecisionTree more adaptively
* [SPARK-3736] Workers reconnect when disassociated from the master.mcheah2014-10-203-20/+72
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Before, if the master node is killed and restarted, the worker nodes would not attempt to reconnect to the Master. Therefore, when the Master node was restarted, the worker nodes needed to be restarted as well. Now, when the Master node is disconnected, the worker nodes will continuously ping the master node in attempts to reconnect to it. Once the master node restarts, it will detect one of the registration requests from its former workers. The result is that the cluster re-enters a healthy state. In addition, when the master does not receive a heartbeat from the worker, the worker was removed; however, when the worker sent a heartbeat to the master, the master used to ignore the heartbeat. Now, a master that receives a heartbeat from a worker that had been disconnected will request the worker to re-attempt the registration process, at which point the worker will send a RegisterWorker request and be re-connected accordingly. Re-connection attempts per worker are submitted every N seconds, where N is configured by the property spark.worker.reconnect.interval - this has a default of 60 seconds right now. Author: mcheah <mcheah@palantir.com> Closes #2828 from mccheah/reconnect-dead-workers and squashes the following commits: 83f8bc9 [mcheah] [SPARK-3736] More informative log message, and fixing some indentation. fe0e02f [mcheah] [SPARK-3736] Moving reconnection logic to registerWithMaster(). 94ddeca [mcheah] [SPARK-3736] Changing a log warning to a log info. a698e35 [mcheah] [SPARK-3736] Addressing PR comment to make some defs private. b9a3077 [mcheah] [SPARK-3736] Addressing PR comments related to reconnection. 2ad5ed5 [mcheah] [SPARK-3736] Cancel attempts to reconnect if the master changes. b5b34af [mcheah] [SPARK-3736] Workers reconnect when disassociated from the master.
* [SPARK-3986][SQL] Fix package names to fit their directory names.Takuya UESHIN2014-10-202-4/+2
| | | | | | | | | | | | Package names of 2 test suites are different from their directory names. - `GeneratedEvaluationSuite` - `GeneratedMutableEvaluationSuite` Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #2835 from ueshin/issues/SPARK-3986 and squashes the following commits: fa2cc05 [Takuya UESHIN] Fix package names to fit their directory names.
* [SPARK-4010][Web UI]Spark UI returns 500 in yarn-client modeGuoQiang Li2014-10-202-5/+5
| | | | | | | | | | | The problem caused by #1966 CC YanTangZhai andrewor14 Author: GuoQiang Li <witgo@qq.com> Closes #2858 from witgo/SPARK-4010 and squashes the following commits: 9866fbf [GuoQiang Li] Spark UI returns 500 in yarn-client mode
* [SPARK-3948][Shuffle]Fix stream corruption bug in sort-based shufflejerryshao2014-10-202-6/+28
| | | | | | | | | | | | Kernel 2.6.32 bug will lead to unexpected behavior of transferTo in copyStream, and this will corrupt the shuffle output file in sort-based shuffle, which will somehow introduce PARSING_ERROR(2), deserialization error or offset out of range. Here fix this by adding append flag, also add some position checking code. Details can be seen in [SPARK-3948](https://issues.apache.org/jira/browse/SPARK-3948). Author: jerryshao <saisai.shao@intel.com> Closes #2824 from jerryshao/SPARK-3948 and squashes the following commits: be0533a [jerryshao] Address the comments a82b184 [jerryshao] add configuration to control the NIO way of copying stream e17ada2 [jerryshao] Fix kernel 2.6.32 bug led unexpected behavior of transferTo
* [SPARK-3902] [SPARK-3590] Stabilize AsynRDDActions and add Java APIJosh Rosen2014-10-196-35/+246
| | | | | | | | | | | | | | | | | | This PR adds a Java API for AsyncRDDActions and promotes the API from `Experimental` to stable. Author: Josh Rosen <joshrosen@apache.org> Author: Josh Rosen <joshrosen@databricks.com> Closes #2760 from JoshRosen/async-rdd-actions-in-java and squashes the following commits: 0d45fbc [Josh Rosen] Whitespace fix. ad3ae53 [Josh Rosen] Merge remote-tracking branch 'origin/master' into async-rdd-actions-in-java c0153a5 [Josh Rosen] Remove unused variable. e8e2867 [Josh Rosen] Updates based on Marcelo's review feedback 7a1417f [Josh Rosen] Removed unnecessary java.util import. 6f8f6ac [Josh Rosen] Fix import ordering. ff28e49 [Josh Rosen] Add MiMa excludes and fix a scalastyle error. 346e46e [Josh Rosen] [SPARK-3902] Stabilize AsyncRDDActions; add Java API.
* [SPARK-2546] Clone JobConf for each task (branch-1.0 / 1.1 backport)Josh Rosen2014-10-192-15/+47
| | | | | | | | | | | | | | | | | | | | This patch attempts to fix SPARK-2546 in `branch-1.0` and `branch-1.1`. The underlying problem is that thread-safety issues in Hadoop Configuration objects may cause Spark tasks to get stuck in infinite loops. The approach taken here is to clone a new copy of the JobConf for each task rather than sharing a single copy between tasks. Note that there are still Configuration thread-safety issues that may affect the driver, but these seem much less likely to occur in practice and will be more complex to fix (see discussion on the SPARK-2546 ticket). This cloning is guarded by a new configuration option (`spark.hadoop.cloneConf`) and is disabled by default in order to avoid unexpected performance regressions for workloads that are unaffected by the Configuration thread-safety issues. Author: Josh Rosen <joshrosen@apache.org> Closes #2684 from JoshRosen/jobconf-fix-backport and squashes the following commits: f14f259 [Josh Rosen] Add configuration option to control cloning of Hadoop JobConf. b562451 [Josh Rosen] Remove unused jobConfCacheKey field. dd25697 [Josh Rosen] [SPARK-2546] [1.0 / 1.1 backport] Clone JobConf for each task. (cherry picked from commit 2cd40db2b3ab5ddcb323fd05c171dbd9025f9e71) Signed-off-by: Josh Rosen <joshrosen@databricks.com> Conflicts: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
* [SPARK-3952] [Streaming] [PySpark] add Python examples in Streaming ↵Davies Liu2014-10-185-14/+391
| | | | | | | | | | | | | | | | | | Programming Guide Having Python examples in Streaming Programming Guide. Also add RecoverableNetworkWordCount example. Author: Davies Liu <davies.liu@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #2808 from davies/pyguide and squashes the following commits: 8d4bec4 [Davies Liu] update readme 26a7e37 [Davies Liu] fix format 3821c4d [Davies Liu] address comments, add missing file 7e4bb8a [Davies Liu] add Python examples in Streaming Programming Guide
* SPARK-3926 [CORE] Result of JavaRDD.collectAsMap() is not SerializableSean Owen2014-10-184-9/+23
| | | | | | | | | | | | | Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are Author: Sean Owen <sowen@cloudera.com> Closes #2805 from srowen/SPARK-3926 and squashes the following commits: ecb78ee [Sean Owen] Fix conflict between java.io.Serializable and use of Scala's Serializable f4717f9 [Sean Owen] Oops, fix compile problem ae1b36f [Sean Owen] Expand to cover Maps returned from other Java API methods as well 51c26c2 [Sean Owen] Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are
* [SPARK-3934] [SPARK-3918] [mllib] Bug fixes for RandomForest, DecisionTreeJoseph K. Bradley2014-10-176-20/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SPARK-3934: When run with a mix of unordered categorical and continuous features, on multiclass classification, RandomForest fails. The bug is in the sanity checks in getFeatureOffset and getLeftRightFeatureOffsets, which use the wrong indices for checking whether features are unordered. Fix: Remove the sanity checks since they are not really needed, and since they would require DTStatsAggregator to keep track of an extra set of indices (for the feature subset). Added test to RandomForestSuite which failed with old version but now works. SPARK-3918: Added baggedInput.unpersist at end of training. Also: * I removed DTStatsAggregator.isUnordered since it is no longer used. * DecisionTreeMetadata: Added logWarning when maxBins is automatically reduced. * Updated DecisionTreeRunner to explicitly fix the test data to have the same number of features as the training data. This is a temporary fix which should eventually be replaced by pre-indexing both datasets. * RandomForestModel: Updated toString to print total number of nodes in forest. * Changed Predict class to be public DeveloperApi. This was necessary to allow users to create their own trees by hand (for testing). CC: mengxr manishamde chouqin codedeft Just notifying you of these small bug fixes. Author: Joseph K. Bradley <joseph.kurata.bradley@gmail.com> Closes #2785 from jkbradley/dtrunner-update and squashes the following commits: 9132321 [Joseph K. Bradley] merged with master, fixed imports 9dbd000 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update e116473 [Joseph K. Bradley] Changed Predict class to be public DeveloperApi. f502e65 [Joseph K. Bradley] bug fix for SPARK-3934 7f3d60f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update ba567ab [Joseph K. Bradley] Changed DTRunner to load test data using same number of features as in training data. 4e88c1f [Joseph K. Bradley] changed RF toString to print total number of nodes
* [SPARK-3985] [Examples] fix file path using os.path.joinDaoyuan Wang2014-10-171-1/+1
| | | | | | | | Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #2834 from adrian-wang/sqlpypath and squashes the following commits: da7aa95 [Daoyuan Wang] fix file path using path.join
* [SPARK-3855][SQL] Preserve the result attribute of python UDFs though ↵Michael Armbrust2014-10-173-3/+17
| | | | | | | | | | | | | | transformations In the current implementation it was possible for the reference to change after analysis. Author: Michael Armbrust <michael@databricks.com> Closes #2717 from marmbrus/pythonUdfResults and squashes the following commits: da14879 [Michael Armbrust] Fix test 6343bcb [Michael Armbrust] add test 9533286 [Michael Armbrust] Correctly preserve the result attribute of python UDFs though transformations
* [SPARK-3979] [yarn] Use fs's default replication.Marcelo Vanzin2014-10-171-1/+2
| | | | | | | | | | | | | | | This avoids issues when HDFS is configured in a way that would not allow the hardcoded default replication of "3". Note: getDefaultReplication(Path) was added in 0.23.3, and the oldest one available on Maven Central is 0.23.7, so I chose to not add code to access that method via reflection. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #2831 from vanzin/SPARK-3979 and squashes the following commits: b0e3a97 [Marcelo Vanzin] [SPARK-3979] [yarn] Use fs's default replication.
* [SPARK-3935][Core] log the number of records that has been writtenlikun2014-10-171-2/+0
| | | | | | | | | | | | There is a unused variable(count) in saveAsHadoopDataset in PairRDDFunctions.scala. The initial idea of this variable seems to count the number of records, so I am adding a log statement to log the number of records that has been written to the writer. Author: likun <jacky.likun@huawei.com> Author: jackylk <jacky.likun@huawei.com> Closes #2791 from jackylk/SPARK-3935 and squashes the following commits: a874047 [jackylk] removing the unused variable in PairRddFunctions.scala 3bf43c7 [likun] log the number of records has been written
* [SPARK-3973] Print call site information for broadcastsShivaram Venkataraman2014-10-161-0/+2
| | | | | | | | | | Its hard to debug which broadcast variables refer to what in a big codebase. Printing call site information helps in debugging. Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #2829 from shivaram/spark-broadcast-print and squashes the following commits: cd6dbdf [Shivaram Venkataraman] Print call site information for broadcasts
* [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section ↵yantangzhai2014-10-161-1/+4
| | | | | | | | | | | | | | | | | | | | sometimes JobProgressPage could not show Fair Scheduler Pools section sometimes. SparkContext starts webui and then postEnvironmentUpdate. Sometimes JobProgressPage is accessed between webui starting and postEnvironmentUpdate, then the lazy val isFairScheduler will be false. The Fair Scheduler Pools section will not display any more. Author: yantangzhai <tyz0303@163.com> Author: YanTangZhai <hakeemzhai@tencent.com> Closes #1966 from YanTangZhai/SPARK-3067 and squashes the following commits: d4323f8 [yantangzhai] update [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes 8a00106 [YanTangZhai] Merge pull request #6 from apache/master b6391cc [yantangzhai] revert [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes d2226cd [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cbcba66 [YanTangZhai] Merge pull request #3 from apache/master aac7f7b [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cdef539 [YanTangZhai] Merge pull request #1 from apache/master
* [SPARK-3741] Add afterExecute for handleConnectExecutorzsxwing2014-10-161-1/+10
| | | | | | | | | | Sorry. I found that I forgot to add `afterExecute` for `handleConnectExecutor` in #2593. Author: zsxwing <zsxwing@gmail.com> Closes #2794 from zsxwing/SPARK-3741 and squashes the following commits: a0bc4dd [zsxwing] Add afterExecute for handleConnectExecutor
* [SPARK-3890][Docs]remove redundant spark.executor.memory in docWangTaoTheTonic2014-10-161-12/+4
| | | | | | | | | | | | | Introduced in https://github.com/pwendell/spark/commit/f7e79bc42c1635686c3af01eef147dae92de2529, I'm not sure why we need two spark.executor.memory here. Author: WangTaoTheTonic <barneystinson@aliyun.com> Author: WangTao <barneystinson@aliyun.com> Closes #2745 from WangTaoTheTonic/redundantconfig and squashes the following commits: e7564dc [WangTao] too long line fdbdb1f [WangTaoTheTonic] trivial workaround d06b6e5 [WangTaoTheTonic] remove redundant spark.executor.memory in doc
* [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfoZhang, Liye2014-10-161-4/+5
| | | | | | | | | | | In BlockManagermasterActor, _remainingMem would increase memSize for twice when updateBlockInfo if new storageLevel is invalid and old storageLevel is "useMemory". Also, _remainingMem should increase with original memory size instead of new memSize. Author: Zhang, Liye <liye.zhang@intel.com> Closes #2792 from liyezhang556520/spark-3941-remainMem and squashes the following commits: 3d487cc [Zhang, Liye] make the code concise 0380a32 [Zhang, Liye] [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfo
* [SQL]typo in HiveFromSparkKun Li2014-10-161-1/+1
| | | | | | | | Author: Kun Li <jacky.likun@gmail.com> Closes #2809 from jackylk/patch-1 and squashes the following commits: 46c926b [Kun Li] typo in HiveFromSpark
* [SPARK-3923] Increase Akka heartbeat pause above heartbeat intervalAaron Davidson2014-10-162-2/+2
| | | | | | | | | | | | | Something about the 2.3.4 upgrade seems to have made the issue manifest where all the services disconnect from each other after exactly 1000 seconds (which is the heartbeat interval). [This post](https://groups.google.com/forum/#!topic/akka-user/X3xzpTCbEFs) suggests that heartbeat pause should be greater than heartbeat interval, and increasing the pause from 600s to 6000s seems to have rectified the issue. My current cluster has now exceeded 1400s of uptime without failure! I do not know why this fixed it, because the threshold we have set for the failure detector is the exponent of a timeout, and 300 is extremely large. Perhaps the default failure detector changed in 2.3.4 and now ignores threshold. Author: Aaron Davidson <aaron@databricks.com> Closes #2784 from aarondav/fix-timeout and squashes the following commits: bd1151a [Aaron Davidson] Increase pause, don't decrease interval 9cb0372 [Aaron Davidson] [SPARK-3923] Decrease Akka heartbeat interval below heartbeat pause
* SPARK-3874: Provide stable TaskContext APIPrashant Sharma2014-10-1616-223/+186
| | | | | | | | | | | | | | | | | | This is a small number of clean-up changes on top of #2782. Closes #2782. Author: Prashant Sharma <prashant.s@imaginea.com> Author: Patrick Wendell <pwendell@gmail.com> Closes #2803 from pwendell/pr-2782 and squashes the following commits: 56d5b7a [Patrick Wendell] Minor clean-up 44089ec [Patrick Wendell] Clean-up the TaskContext API. ed551ce [Prashant Sharma] Fixed a typo df261d0 [Prashant Sharma] Josh's suggestion facf3b1 [Prashant Sharma] Fixed the mima issue. 7ecc2fe [Prashant Sharma] CR, Moved implementations to TaskContextImpl bbd9e05 [Prashant Sharma] adding missed out files to git. ef633f5 [Prashant Sharma] SPARK-3874, Provide stable TaskContext API
* [SQL] Fixes the race condition that may cause test failureCheng Lian2014-10-161-6/+0
| | | | | | | | | | | | The removed `Future` was used to end the test case as soon as the Spark SQL CLI process exits. When the process exits prematurely, this mechanism prevents the test case to wait until timeout. But it also creates a race condition: when `foundAllExpectedAnswers.tryFailure` is called, there are chances that the last expected output line of the CLI process hasn't been caught by the main logics of the test code, thus fails the test case. Removing this `Future` doesn't affect correctness. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #2823 from liancheng/clean-clisuite and squashes the following commits: 489a97c [Cheng Lian] Fixes the race condition that may cause test failure
* [SPARK-3971] [MLLib] [PySpark] hotfix: Customized pickler should work in ↵Davies Liu2014-10-1614-34/+101
| | | | | | | | | | | | | | | | cluster mode Customized pickler should be registered before unpickling, but in executor, there is no way to register the picklers before run the tasks. So, we need to register the picklers in the tasks itself, duplicate the javaToPython() and pythonToJava() in MLlib, call SerDe.initialize() before pickling or unpickling. Author: Davies Liu <davies.liu@gmail.com> Closes #2830 from davies/fix_pickle and squashes the following commits: 0c85fb9 [Davies Liu] revert the privacy change 6b94e15 [Davies Liu] use JavaConverters instead of JavaConversions 0f02050 [Davies Liu] hotfix: Customized pickler does not work in cluster
* [SPARK-3944][Core] Code re-factored as suggestedShiti2014-10-161-8/+8
| | | | | | | | Author: Shiti <ssaxena.ece@gmail.com> Closes #2810 from Shiti/master and squashes the following commits: 051d82f [Shiti] setting the default value of uri scheme to "file" where matching "file" or None yields the same result
* [Core] Upgrading ScalaStyle version to 0.5 and removing ↵prudhvi2014-10-163-60/+2
| | | | | | | | | | SparkSpaceAfterCommentStartChecker. Author: prudhvi <prudhvi953@gmail.com> Closes #2799 from prudhvije/ScalaStyle/space-after-comment-start and squashes the following commits: fc263a1 [prudhvi] [Core] Using scalastyle to check the space after comment start
* [SPARK-2098] All Spark processes should support spark-defaults.conf, config fileGuoQiang Li2014-10-148-50/+124
| | | | | | | | | | | | | This is another implementation about #1256 cc andrewor14 vanzin Author: GuoQiang Li <witgo@qq.com> Closes #2379 from witgo/SPARK-2098-new and squashes the following commits: 4ef1cbd [GuoQiang Li] review commit 49ef70e [GuoQiang Li] Refactor getDefaultPropertiesFile c45d20c [GuoQiang Li] All Spark processes should support spark-defaults.conf, config file
* SPARK-1307 [DOCS] Don't use term 'standalone' to refer to a Spark ApplicationSean Owen2014-10-145-36/+37
| | | | | | | | | | | | HT to Diana, just proposing an implementation of her suggestion, which I rather agreed with. Is there a second/third for the motion? Refer to "self-contained" rather than "standalone" apps to avoid confusion with standalone deployment mode. And fix placement of reference to this in MLlib docs. Author: Sean Owen <sowen@cloudera.com> Closes #2787 from srowen/SPARK-1307 and squashes the following commits: b5b82e2 [Sean Owen] Refer to "self-contained" rather than "standalone" apps to avoid confusion with standalone deployment mode. And fix placement of reference to this in MLlib docs.
* [SPARK-3943] Some scripts bin\*.cmd pollutes environment variables in WindowsMasayoshi TSUZUKI2014-10-146-289/+342
| | | | | | | | | | | | Modified not to pollute environment variables. Just moved the main logic into `XXX2.cmd` from `XXX.cmd`, and call `XXX2.cmd` with cmd command in `XXX.cmd`. `pyspark.cmd` and `spark-class.cmd` are already using the same way, but `spark-shell.cmd`, `spark-submit.cmd` and `/python/docs/make.bat` are not. Author: Masayoshi TSUZUKI <tsudukim@oss.nttdata.co.jp> Closes #2797 from tsudukim/feature/SPARK-3943 and squashes the following commits: b397a7d [Masayoshi TSUZUKI] [SPARK-3943] Some scripts bin\*.cmd pollutes environment variables in Windows
* [SPARK-3869] ./bin/spark-class miss Java version with _JAVA_OPTIONS setcocoatomo2014-10-141-1/+1
| | | | | | | | | | | When _JAVA_OPTIONS environment variable is set, a command "java -version" outputs a message like "Picked up _JAVA_OPTIONS: -Dfile.encoding=UTF-8". ./bin/spark-class knows java version from the first line of "java -version" output, so it mistakes java version with _JAVA_OPTIONS set. Author: cocoatomo <cocoatomo77@gmail.com> Closes #2725 from cocoatomo/issues/3869-mistake-java-version and squashes the following commits: f894ebd [cocoatomo] [SPARK-3869] ./bin/spark-class miss Java version with _JAVA_OPTIONS set
* SPARK-3803 [MLLIB] ArrayIndexOutOfBoundsException found in executing ↵Sean Owen2014-10-141-7/+15
| | | | | | | | | | | | computePrincipalComponents Avoid overflow in computing n*(n+1)/2 as much as possible; throw explicit error when Gramian computation will fail due to negative array size; warn about large result when computing Gramian too Author: Sean Owen <sowen@cloudera.com> Closes #2801 from srowen/SPARK-3803 and squashes the following commits: b4e6d92 [Sean Owen] Avoid overflow in computing n*(n+1)/2 as much as possible; throw explicit error when Gramian computation will fail due to negative array size; warn about large result when computing Gramian too
* [SPARK-3944][Core] Using Option[String] where value of String can be nullshitis2014-10-141-12/+14
| | | | | | | | Author: shitis <ssaxena.ece@gmail.com> Closes #2795 from Shiti/master and squashes the following commits: 46897d7 [shitis] Using Option Wrapper to convert String with value null to None
* [SPARK-3946] gitignore in /python includes wrong directoryMasayoshi TSUZUKI2014-10-141-1/+1
| | | | | | | | | | Modified to ignore not the docs/ directory, but only the docs/_build/ which is the output directory of sphinx build. Author: Masayoshi TSUZUKI <tsudukim@oss.nttdata.co.jp> Closes #2796 from tsudukim/feature/SPARK-3946 and squashes the following commits: 2bea6a9 [Masayoshi TSUZUKI] [SPARK-3946] gitignore in /python includes wrong directory
* SPARK-3178 setting SPARK_WORKER_MEMORY to a value without a label (m or g) ↵Bill Bejeck2014-10-142-2/+93
| | | | | | | | | | | | sets the worker memory limit to zero Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label. Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen). Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable. Author: Bill Bejeck <bbejeck@gmail.com> Closes #2309 from bbejeck/spark-memory-worker and squashes the following commits: 51cf915 [Bill Bejeck] SPARK-3178 - Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label. Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen). Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable.
* [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone modeAaron Davidson2014-10-135-12/+15
| | | | | | | | | | | | | | | The goal of this patch is to fix the swapped arguments in standalone mode, which was caused by https://github.com/apache/spark/commit/79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-79391110e9f26657e415aa169a004998R153. More details can be found in the JIRA: [SPARK-3921](https://issues.apache.org/jira/browse/SPARK-3921) Tested in Standalone mode, but not in Mesos. Author: Aaron Davidson <aaron@databricks.com> Closes #2779 from aarondav/fix-standalone and squashes the following commits: 725227a [Aaron Davidson] Fix ExecutorRunnerTest 9d703fe [Aaron Davidson] [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone mode
* [SPARK-3912][Streaming] Fixed flakyFlumeStreamSuiteTathagata Das2014-10-131-64/+102
| | | | | | | | | | | | | | | | | @harishreedharan @pwendell See JIRA for diagnosis of the problem https://issues.apache.org/jira/browse/SPARK-3912 The solution was to reimplement it. 1. Find a free port (by binding and releasing a server-scoket), and then use that port 2. Remove thread.sleep()s, instead repeatedly try to create a sender and send data and check whether data was sent. Use eventually() to minimize waiting time. 3. Check whether all the data was received, without caring about batches. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #2773 from tdas/flume-test-fix and squashes the following commits: 93cd7f6 [Tathagata Das] Reimplimented FlumeStreamSuite to be more robust.
* [SPARK-3809][SQL] Fixes test suites in hive-thriftserverCheng Lian2014-10-132-39/+60
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | As scwf pointed out, `HiveThriftServer2Suite` isn't effective anymore after the Thrift server was made a daemon. On the other hand, these test suites were known flaky, PR #2214 tried to fix them but failed because of unknown Jenkins build error. This PR fixes both sets of issues. In this PR, instead of watching `start-thriftserver.sh` output, the test code start a `tail` process to watch the log file. A `Thread.sleep` has to be introduced because the `kill` command used in `stop-thriftserver.sh` is not synchronous. As for the root cause of the mysterious Jenkins build failure. Please refer to [this comment](https://github.com/apache/spark/pull/2675#issuecomment-58464189) below for details. ---- (Copied from PR description of #2214) This PR fixes two issues of `HiveThriftServer2Suite` and brings 1 enhancement: 1. Although metastore, warehouse directories and listening port are randomly chosen, all test cases share the same configuration. Due to parallel test execution, one of the two test case is doomed to fail 2. We caught any exceptions thrown from a test case and print diagnosis information, but forgot to re-throw the exception... 3. When the forked server process ends prematurely (e.g., fails to start), the `serverRunning` promise is completed with a failure, preventing the test code to keep waiting until timeout. So, embarrassingly, this test suite was failing continuously for several days but no one had ever noticed it... Fortunately no bugs in the production code were covered under the hood. Author: Cheng Lian <lian.cs.zju@gmail.com> Author: wangfei <wangfei1@huawei.com> Closes #2675 from liancheng/fix-thriftserver-tests and squashes the following commits: 1c384b7 [Cheng Lian] Minor code cleanup, restore the logging level hack in TestHive.scala 7805c33 [wangfei] reset SPARK_TESTING to avoid loading Log4J configurations in testing class paths af2b5a9 [Cheng Lian] Removes log level hacks from TestHiveContext d116405 [wangfei] make sure that log4j level is INFO ee92a82 [Cheng Lian] Relaxes timeout 7fd6757 [Cheng Lian] Fixes test suites in hive-thriftserver
* [SQL]Small bug in unresolved.scalaLiquan Pei2014-10-131-1/+1
| | | | | | | | | | name should throw exception with name instead of exprId. Author: Liquan Pei <liquanpei@gmail.com> Closes #2758 from Ishiihara/SparkSQL-bug and squashes the following commits: aa36a3b [Liquan Pei] small bug
* SPARK-3807: SparkSql does not work for tables created using custom serdechirag2014-10-133-2/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SparkSql crashes on selecting tables using custom serde. Example: ---------------- CREATE EXTERNAL TABLE table_name PARTITIONED BY ( a int) ROW FORMAT 'SERDE "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" with serdeproperties("serialization.format"="org.apache.thrift.protocol.TBinaryProtocol","serialization.class"="ser_class") STORED AS SEQUENCEFILE; The following exception is seen on running a query like 'select * from table_name limit 1': ERROR CliDriver: org.apache.hadoop.hive.serde2.SerDeException: java.lang.NullPointerException at org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer.initialize(ThriftDeserializer.java:68) at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializer(TableDesc.java:80) at org.apache.spark.sql.hive.execution.HiveTableScan.addColumnMetadataToConf(HiveTableScan.scala:86) at org.apache.spark.sql.hive.execution.HiveTableScan.<init>(HiveTableScan.scala:100) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.SQLContext$SparkPlanner.pruneFilterProject(SQLContext.scala:364) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$.apply(HiveStrategies.scala:184) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:280) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:402) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:400) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:406) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:406) at org.apache.spark.sql.hive.HiveContext$QueryExecution.stringResult(HiveContext.scala:406) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:59) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:291) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:226) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source) at java.lang.reflect.Method.invoke(Unknown Source) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:328) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:75) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.NullPointerException Author: chirag <chirag.aggarwal@guavus.com> Closes #2674 from chiragaggarwal/branch-1.1 and squashes the following commits: 370c31b [chirag] SPARK-3807: Add a test case to validate the fix. 1f26805 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (Incorporated Review Comments) ba4bc0c [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde 5c73b72 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (cherry picked from commit 925e22d3132b983a2fcee31e3878b680c7ff92da) Signed-off-by: Michael Armbrust <michael@databricks.com>
* [SQL] Add type checking debugging functionsMichael Armbrust2014-10-138-20/+163
| | | | | | | | | | | | | Adds some functions that were very useful when trying to track down the bug from #2656. This change also changes the tree output for query plans to include the `'` prefix to unresolved nodes and `!` prefix to nodes that refer to non-existent attributes. Author: Michael Armbrust <michael@databricks.com> Closes #2657 from marmbrus/debugging and squashes the following commits: 654b926 [Michael Armbrust] Clean-up, add tests 763af15 [Michael Armbrust] Add typeChecking debugging functions 8c69303 [Michael Armbrust] Add inputSet, references to QueryPlan. Improve tree string with a prefix to denote invalid or unresolved nodes. fbeab54 [Michael Armbrust] Better toString, factories for AttributeSet.