diff --git a/CHANGES.txt b/CHANGES.txt index 3693d59947ce7..dc643ed184d9b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,6 +1,684 @@ Spark Change Log ---------------- +Release 1.1.1 + + [SPARK-4295][External]Fix exception in SparkSinkSuite + maji2014 + 2014-11-11 02:18:27 -0800 + Commit: bf867c3, github.com/apache/spark/pull/3177 + + [branch-1.1][SPARK-3990] add a note on ALS usage + Xiangrui Meng + 2014-11-10 22:39:09 -0800 + Commit: b2cb357, github.com/apache/spark/pull/3190 + + [BRANCH-1.1][SPARK-2652] change the default spark.serializer in pyspark back to Kryo + Xiangrui Meng + 2014-11-10 22:21:14 -0800 + Commit: 11798d0, github.com/apache/spark/pull/3187 + + [SPARK-4330][Doc] Link to proper URL for YARN overview + Kousuke Saruta + 2014-11-10 22:18:00 -0800 + Commit: d313be8, github.com/apache/spark/pull/3196 + + [SQL] Backport backtick and smallint JDBC fixes to 1.1 + Michael Armbrust , ravipesala , scwf + 2014-11-10 19:51:07 -0800 + Commit: 8a1d818, github.com/apache/spark/pull/3199 + + Update versions for 1.1.1 release + Andrew Or + 2014-11-10 18:40:34 -0800 + Commit: 01d233e + + [SPARK-3495][SPARK-3496] Backporting block replication fixes made in master to branch 1.1 + Tathagata Das + 2014-11-10 18:23:02 -0800 + Commit: be0cc99, github.com/apache/spark/pull/3191 + + [SPARK-3954][Streaming] Optimization to FileInputDStream + surq + 2014-11-10 17:37:16 -0800 + Commit: 3d889df, github.com/apache/spark/pull/2811 + + [SPARK-3971][SQL] Backport #2843 to branch-1.1 + Cheng Lian , Cheng Lian , Michael Armbrust + 2014-11-10 17:04:10 -0800 + Commit: 64945f8, github.com/apache/spark/pull/3113 + + [SPARK-4308][SQL] Follow up of #3175 for branch 1.1 + Cheng Lian + 2014-11-10 16:57:34 -0800 + Commit: b3ef06b, github.com/apache/spark/pull/3176 + + [SPARK-2548][HOTFIX][Streaming] Removed use of o.a.s.streaming.Durations in branch 1.1 + Tathagata Das + 2014-11-10 14:13:42 -0800 + Commit: 86b1bd0, github.com/apache/spark/pull/3188 + + Update RecoverableNetworkWordCount.scala + comcmipi + 2014-11-10 12:33:48 -0800 + Commit: 254b135, github.com/apache/spark/pull/2735 + + SPARK-2548 [STREAMING] JavaRecoverableWordCount is missing + Sean Owen + 2014-11-10 11:47:27 -0800 + Commit: cdcf546, github.com/apache/spark/pull/2564 + + [SPARK-4169] [Core] Accommodate non-English Locales in unit tests + Niklas Wilcke <1wilcke@informatik.uni-hamburg.de> + 2014-11-10 11:37:38 -0800 + Commit: dc38def, github.com/apache/spark/pull/3036 + + [SPARK-4301] StreamingContext should not allow start() to be called after calling stop() + Josh Rosen + 2014-11-08 18:10:23 -0800 + Commit: 78cd3ab, github.com/apache/spark/pull/3160 + + [SPARK-4304] [PySpark] Fix sort on empty RDD + Davies Liu + 2014-11-07 20:53:03 -0800 + Commit: 4895f65, github.com/apache/spark/pull/3162 + + Update JavaCustomReceiver.java + xiao321 <1042460381@qq.com> + 2014-11-07 12:56:49 -0800 + Commit: 4fb26df, github.com/apache/spark/pull/3153 + + [SPARK-4249][GraphX]fix a problem of EdgePartitionBuilder in Graphx + lianhuiwang + 2014-11-06 10:46:45 -0800 + Commit: 0a40eac, github.com/apache/spark/pull/3138 + + [SPARK-4158] Fix for missing resources. + Brenden Matthews + 2014-11-05 16:02:44 -0800 + Commit: c58c1bb, github.com/apache/spark/pull/3024 + + SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i... + Jongyoul Lee + 2014-11-05 15:49:42 -0800 + Commit: 590a943, github.com/apache/spark/pull/3034 + + [branch-1.1][SPARK-4148][PySpark] fix seed distribution and add some tests for rdd.sample + Xiangrui Meng + 2014-11-05 10:30:10 -0800 + Commit: 44751af, github.com/apache/spark/pull/3104 + + [SPARK-4115][GraphX] Add overrided count for edge counting of EdgeRDD. + luluorta + 2014-11-01 01:22:46 -0700 + Commit: 1b282cd, github.com/apache/spark/pull/2975 + + [SPARK-4097] Fix the race condition of 'thread' + zsxwing + 2014-10-29 14:42:50 -0700 + Commit: abdb90b, github.com/apache/spark/pull/2957 + + [SPARK-4065] Add check for IPython on Windows + Michael Griffiths + 2014-10-28 12:47:21 -0700 + Commit: f0c5717, github.com/apache/spark/pull/2910 + + [SPARK-4107] Fix incorrect handling of read() and skip() return values (branch-1.1 backport) + Josh Rosen + 2014-10-28 12:30:12 -0700 + Commit: 286f1ef, github.com/apache/spark/pull/2974 + + [SPARK-4110] Wrong comments about default settings in spark-daemon.sh + Kousuke Saruta + 2014-10-28 12:29:01 -0700 + Commit: dee3317, github.com/apache/spark/pull/2972 + + [MLlib] SPARK-3987: add test case on objective value for NNLS + coderxiang + 2014-10-27 19:43:39 -0700 + Commit: 2ef2f5a, github.com/apache/spark/pull/2965 + + Fix build breakage introduced by 6c10c2770c718287f9cc2af4109b701fa1057b70 + Josh Rosen + 2014-10-25 20:33:17 -0700 + Commit: 2eb9d7c + + Revert "[SPARK-4056] Upgrade snappy-java to 1.1.1.5" + Josh Rosen + 2014-10-25 17:09:01 -0700 + Commit: c1989aa + + [SPARK-4056] Upgrade snappy-java to 1.1.1.5 + Josh Rosen , Josh Rosen + 2014-10-24 17:21:08 -0700 + Commit: b7541ae, github.com/apache/spark/pull/2911 + + [SPARK-4080] Only throw IOException from [write|read][Object|External] + Josh Rosen + 2014-10-24 15:06:15 -0700 + Commit: 6c10c27, github.com/apache/spark/pull/2932 + + [SPARK-4006] In long running contexts, we encountered the situation of d... + Tal Sliwowicz + 2014-10-24 13:51:25 -0700 + Commit: 59297e9, github.com/apache/spark/pull/2915 + + [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file + Kousuke Saruta + 2014-10-24 13:08:21 -0700 + Commit: 80dde80, github.com/apache/spark/pull/2925 + + [SPARK-4076] Parameter expansion in spark-config is wrong + Kousuke Saruta + 2014-10-24 13:04:35 -0700 + Commit: 386fc46, github.com/apache/spark/pull/2930 + + [SPARK-2652] [PySpark] donot use KyroSerializer as default serializer + Davies Liu + 2014-10-23 23:58:00 -0700 + Commit: 926f8ca, github.com/apache/spark/pull/2916 + + [SPARK-3426] Fix sort-based shuffle error when spark.shuffle.compress and spark.shuffle.spill.compress settings are different + Josh Rosen + 2014-10-22 14:49:58 -0700 + Commit: 5e191fa, github.com/apache/spark/pull/2890 + + [SPARK-3877][YARN] Throw an exception when application is not successful so that the exit code wil be set to 1 (for branch-1.1) + zsxwing + 2014-10-22 15:08:28 -0700 + Commit: eb62094, github.com/apache/spark/pull/2748 + + [SPARK-4010][Web UI]Spark UI returns 500 in yarn-client mode + GuoQiang Li + 2014-10-20 11:01:26 -0700 + Commit: 457ef59, github.com/apache/spark/pull/2858 + + [SPARK-3948][Shuffle]Fix stream corruption bug in sort-based shuffle + jerryshao + 2014-10-20 10:20:21 -0700 + Commit: 12a61d8, github.com/apache/spark/pull/2824 + + [SPARK-2546] Clone JobConf for each task (branch-1.0 / 1.1 backport) + Josh Rosen + 2014-10-19 00:31:06 -0700 + Commit: 2cd40db, github.com/apache/spark/pull/2684 + + SPARK-3926 [CORE] Result of JavaRDD.collectAsMap() is not Serializable + Sean Owen + 2014-10-18 12:33:20 -0700 + Commit: 327404d, github.com/apache/spark/pull/2805 + + [SPARK-3606] [yarn] Correctly configure AmIpFilter for Yarn HA (1.1 vers... + Marcelo Vanzin + 2014-10-17 00:53:15 -0700 + Commit: 0d958f1, github.com/apache/spark/pull/2497 + + [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes + yantangzhai , YanTangZhai + 2014-10-16 19:25:37 -0700 + Commit: 35875e9, github.com/apache/spark/pull/1966 + + [SPARK-3890][Docs]remove redundant spark.executor.memory in doc + WangTaoTheTonic , WangTao + 2014-10-16 19:12:39 -0700 + Commit: 2c41170, github.com/apache/spark/pull/2745 + + [SQL]typo in HiveFromSpark + Kun Li + 2014-10-16 19:00:10 -0700 + Commit: 61e5903, github.com/apache/spark/pull/2809 + + SPARK-3807: SparkSql does not work for tables created using custom serde + chirag + 2014-10-13 13:47:26 -0700 + Commit: 925e22d, github.com/apache/spark/pull/2674 + + [SPARK-3899][Doc]fix wrong links in streaming doc + w00228970 , wangfei + 2014-10-12 23:35:50 -0700 + Commit: 4fc6638, github.com/apache/spark/pull/2749 + + [SPARK-3905][Web UI]The keys for sorting the columns of Executor page ,Stage page Storage page are incorrect + GuoQiang Li + 2014-10-12 22:48:54 -0700 + Commit: a36116c, github.com/apache/spark/pull/2763 + + [SPARK-3121] Wrong implementation of implicit bytesWritableConverter + Jakub Dubovský , Dubovsky Jakub + 2014-10-12 22:03:26 -0700 + Commit: 0e32579, github.com/apache/spark/pull/2712 + + SPARK-3716 [GraphX] Update Analytics.scala for partitionStrategy assignment + NamelessAnalyst + 2014-10-12 14:18:55 -0700 + Commit: 5a21e3e, github.com/apache/spark/pull/2569 + + [SPARK-3711][SQL] Optimize where in clause filter queries + Yash Datta + 2014-10-09 12:59:14 -0700 + Commit: 18ef22a, github.com/apache/spark/pull/2561 + + [SPARK-3844][UI] Truncate appName in WebUI if it is too long + Xiangrui Meng + 2014-10-09 00:00:24 -0700 + Commit: 09d6a81, github.com/apache/spark/pull/2707 + + [SPARK-3788] [yarn] Fix compareFs to do the right thing for HDFS namespaces (1.1 version). + Marcelo Vanzin + 2014-10-08 08:51:17 -0500 + Commit: a44af73, github.com/apache/spark/pull/2650 + + [SPARK-3829] Make Spark logo image on the header of HistoryPage as a link to HistoryPage's page #1 + Kousuke Saruta + 2014-10-07 16:54:49 -0700 + Commit: a1f833f, github.com/apache/spark/pull/2690 + + [SPARK-3777] Display "Executor ID" for Tasks in Stage page + zsxwing + 2014-10-07 16:00:22 -0700 + Commit: e8afb73, github.com/apache/spark/pull/2642 + + [SPARK-3731] [PySpark] fix memory leak in PythonRDD + Davies Liu + 2014-10-07 12:20:12 -0700 + Commit: 5531830, github.com/apache/spark/pull/2668 + + [SPARK-3825] Log more detail when unrolling a block fails + Andrew Or + 2014-10-07 12:52:10 -0700 + Commit: 267c7be, github.com/apache/spark/pull/2688 + + [SPARK-3808] PySpark fails to start in Windows + Masayoshi TSUZUKI + 2014-10-07 11:53:22 -0700 + Commit: 3a7875d, github.com/apache/spark/pull/2669 + + [SPARK-3827] Very long RDD names are not rendered properly in web UI + Hossein + 2014-10-07 11:46:26 -0700 + Commit: 82ab4a7, github.com/apache/spark/pull/2687 + + [SPARK-3792][SQL] Enable JavaHiveQLSuite + scwf + 2014-10-05 17:47:20 -0700 + Commit: 964e3aa, github.com/apache/spark/pull/2652 + + SPARK-1656: Fix potential resource leaks + zsxwing + 2014-10-05 09:55:17 -0700 + Commit: c068d90, github.com/apache/spark/pull/577 + + [SPARK-3597][Mesos] Implement `killTask`. + Brenden Matthews + 2014-10-05 09:49:24 -0700 + Commit: d9cf4d0, github.com/apache/spark/pull/2453 + + [SPARK-3774] typo comment in bin/utils.sh + Masayoshi TSUZUKI + 2014-10-03 13:12:37 -0700 + Commit: e4ddede, github.com/apache/spark/pull/2639 + + [SPARK-3775] Not suitable error message in spark-shell.cmd + Masayoshi TSUZUKI + 2014-10-03 13:09:48 -0700 + Commit: f130256, github.com/apache/spark/pull/2640 + + [SPARK-3535][Mesos] Fix resource handling. + Brenden Matthews + 2014-10-03 12:58:04 -0700 + Commit: 6f15097, github.com/apache/spark/pull/2401 + + [SPARK-3696]Do not override the user-difined conf_dir + WangTaoTheTonic + 2014-10-03 10:42:41 -0700 + Commit: d5af9e1, github.com/apache/spark/pull/2541 + + SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR + EugenCepoi + 2014-10-03 10:03:15 -0700 + Commit: 5d991db, github.com/apache/spark/pull/2481 + + [DEPLOY] SPARK-3759: Return the exit code of the driver process + Eric Eijkelenboom + 2014-10-02 18:04:38 -0700 + Commit: 699af62, github.com/apache/spark/pull/2628 + + [SPARK-3755][Core] avoid trying privileged port when request a non-privileged port + scwf + 2014-10-02 17:47:56 -0700 + Commit: 16789f6, github.com/apache/spark/pull/2623 + + [SQL][Docs] Update the output of printSchema and fix a typo in SQL programming guide. + Yin Huai + 2014-10-02 11:37:24 -0700 + Commit: 6869351, github.com/apache/spark/pull/2630 + + SPARK-3638 | Forced a compatible version of http client in kinesis-asl profile + aniketbhatnagar + 2014-10-01 18:31:18 -0700 + Commit: c52c231, github.com/apache/spark/pull/2535 + + Typo error in KafkaWordCount example + Gaspar Munoz + 2014-10-01 13:47:22 -0700 + Commit: 24ee616, github.com/apache/spark/pull/2614 + + [SPARK-3756] [Core]check exception is caused by an address-port collision properly + scwf + 2014-10-01 11:51:30 -0700 + Commit: b4f690d, github.com/apache/spark/pull/2611 + + SPARK-2626 [DOCS] Stop SparkContext in all examples + Sean Owen + 2014-10-01 11:28:22 -0700 + Commit: 13f33cf, github.com/apache/spark/pull/2575 + + [SPARK-3755][Core] Do not bind port 1 - 1024 to server in spark + scwf + 2014-10-01 11:30:29 -0700 + Commit: c8c3b49, github.com/apache/spark/pull/2610 + + [SPARK-3747] TaskResultGetter could incorrectly abort a stage if it cannot get result for a specific task + Reynold Xin + 2014-10-01 00:29:14 -0700 + Commit: a7d2df4, github.com/apache/spark/pull/2599 + + SPARK-3745 - fix check-license to properly download and check jar + shane knapp + 2014-09-30 13:11:25 -0700 + Commit: 06b96d4, github.com/apache/spark/pull/2596 + + [SPARK-3709] Executors don't always report broadcast block removal properly back to the driver (for branch-1.1) + Reynold Xin + 2014-09-30 12:24:58 -0700 + Commit: a8c6e82, github.com/apache/spark/pull/2591 + + [SPARK-3734] DriverRunner should not read SPARK_HOME from submitter's environment + Josh Rosen + 2014-09-29 23:36:10 -0700 + Commit: 48be657, github.com/apache/spark/pull/2586 + + Fixed the condition in StronglyConnectedComponents Issue: SPARK-3635 + oded + 2014-09-29 18:05:53 -0700 + Commit: 85dd513, github.com/apache/spark/pull/2486 + + [graphX] GraphOps: random pick vertex bug + yingjieMiao + 2014-09-29 18:01:27 -0700 + Commit: e5ab113, github.com/apache/spark/pull/2553 + + [SPARK-3032][Shuffle] Fix key comparison integer overflow introduced sorting exception + jerryshao + 2014-09-29 11:25:32 -0700 + Commit: df5a62f, github.com/apache/spark/pull/2514 + + [CORE] Bugfix: LogErr format in DAGScheduler.scala + Zhang, Liye + 2014-09-29 01:13:15 -0700 + Commit: 7d88471, github.com/apache/spark/pull/2572 + + [SPARK-3715][Docs]minor typo + WangTaoTheTonic + 2014-09-28 18:30:13 -0700 + Commit: 004b6fa, github.com/apache/spark/pull/2567 + + Docs : use "--total-executor-cores" rather than "--cores" after spark-shell + CrazyJvm + 2014-09-27 09:41:04 -0700 + Commit: d9d94e0, github.com/apache/spark/pull/2540 + + SPARK-3639 | Removed settings master in examples + aniketbhatnagar + 2014-09-26 09:47:58 -0700 + Commit: d6ed5ab, github.com/apache/spark/pull/2536 + + [SPARK-1853] Show Streaming application code context (file, line number) in Spark Stages UI + Mubarak Seyed , Tathagata Das + 2014-09-23 15:09:12 -0700 + Commit: 505ed6b, github.com/apache/spark/pull/2464 + + [SPARK-3653] Respect SPARK_*_MEMORY for cluster mode + Andrew Or + 2014-09-23 14:00:33 -0700 + Commit: 5bbc621, github.com/apache/spark/pull/2500 + + SPARK-3612. Executor shouldn't quit if heartbeat message fails to reach ... + Sandy Ryza + 2014-09-23 13:44:18 -0700 + Commit: ffd97be, github.com/apache/spark/pull/2487 + + Update docs to use jsonRDD instead of wrong jsonRdd. + Grega Kespret + 2014-09-22 10:13:44 -0700 + Commit: aab0a1d, github.com/apache/spark/pull/2479 + + [MLLib] Fix example code variable name misspelling in MLLib Feature Extraction guide + RJ Nowling + 2014-09-22 09:10:41 -0700 + Commit: 32bb97f, github.com/apache/spark/pull/2459 + + Revert "[SPARK-3595] Respect configured OutputCommitters when calling saveAsHadoopFile" + Patrick Wendell + 2014-09-21 13:07:20 -0700 + Commit: f5bf7de + + [SPARK-3595] Respect configured OutputCommitters when calling saveAsHadoopFile + Ian Hummel + 2014-09-21 13:04:36 -0700 + Commit: 7a76657, github.com/apache/spark/pull/2450 + + [Docs] Fix outdated docs for standalone cluster + andrewor14 , Andrew Or + 2014-09-19 16:02:38 -0700 + Commit: fd88353, github.com/apache/spark/pull/2461 + + [SPARK-2062][GraphX] VertexRDD.apply does not use the mergeFunc + Larry Xiao , Blie Arkansol , Ankur Dave + 2014-09-18 23:32:32 -0700 + Commit: 1687d6b, github.com/apache/spark/pull/1903 + + [Minor Hot Fix] Move a line in SparkSubmit to the right place + Andrew Or + 2014-09-18 17:49:28 -0700 + Commit: cf15b22, github.com/apache/spark/pull/2452 + + [SPARK-3560] Fixed setting spark.jars system property in yarn-cluster mode + Victsm , Min Shen + 2014-09-18 15:58:14 -0700 + Commit: 832dff6, github.com/apache/spark/pull/2449 + + [SPARK-3589][Minor]remove redundant code + WangTaoTheTonic + 2014-09-18 12:07:24 -0700 + Commit: 2b28692, github.com/apache/spark/pull/2445 + + [SPARK-3565]Fix configuration item not consistent with document + WangTaoTheTonic + 2014-09-17 21:59:23 -0700 + Commit: 32f2222, github.com/apache/spark/pull/2427 + + [SPARK-3564][WebUI] Display App ID on HistoryPage + Kousuke Saruta + 2014-09-17 16:31:58 -0700 + Commit: 3f1f974, github.com/apache/spark/pull/2424 + + Docs: move HA subsections to a deeper indentation level + Andrew Ash + 2014-09-17 15:07:57 -0700 + Commit: 0690410, github.com/apache/spark/pull/2402 + + [SQL][DOCS] Improve table caching section + Michael Armbrust + 2014-09-17 12:41:49 -0700 + Commit: 85e7c52, github.com/apache/spark/pull/2434 + + [SPARK-3490] Disable SparkUI for tests (backport into 1.1) + Andrew Or + 2014-09-16 18:23:28 -0700 + Commit: 937de93, github.com/apache/spark/pull/2415 + + [SPARK-3555] Fix UISuite race condition + Andrew Or + 2014-09-16 16:03:20 -0700 + Commit: 856156b, github.com/apache/spark/pull/2418 + + [SQL][DOCS] Improve section on thrift-server + Michael Armbrust + 2014-09-16 11:51:46 -0700 + Commit: 75158a7, github.com/apache/spark/pull/2384 + + [SPARK-3518] Remove wasted statement in JsonProtocol + Kousuke Saruta + 2014-09-15 16:11:41 -0700 + Commit: 99a6c5e, github.com/apache/spark/pull/2380 + + SPARK-3039: Allow spark to be built using avro-mapred for hadoop2 + Bertrand Bossy + 2014-09-14 21:10:17 -0700 + Commit: 78887f9, github.com/apache/spark/pull/1945 + + [SQL] [Docs] typo fixes + Nicholas Chammas + 2014-09-13 12:34:20 -0700 + Commit: 70f93d5, github.com/apache/spark/pull/2367 + + [SPARK-3515][SQL] Moves test suite setup code to beforeAll rather than in constructor + Cheng Lian + 2014-09-12 20:14:09 -0700 + Commit: 44e534e, github.com/apache/spark/pull/2375 + + [SPARK-3500] [SQL] use JavaSchemaRDD as SchemaRDD._jschema_rdd + Davies Liu + 2014-09-12 19:05:39 -0700 + Commit: 9c06c72, github.com/apache/spark/pull/2369 + + [SPARK-3481] [SQL] Eliminate the error log in local Hive comparison test + Cheng Hao + 2014-09-12 11:29:30 -0700 + Commit: 6cbf83c, github.com/apache/spark/pull/2352 + + Revert "[Spark-3490] Disable SparkUI for tests" + Andrew Or + 2014-09-12 10:40:03 -0700 + Commit: f17b795 + + [SPARK-3465] fix task metrics aggregation in local mode + Davies Liu + 2014-09-11 18:53:26 -0700 + Commit: e69deb8, github.com/apache/spark/pull/2338 + + [SPARK-3429] Don't include the empty string "" as a defaultAclUser + Andrew Ash + 2014-09-11 17:28:36 -0700 + Commit: 4245404, github.com/apache/spark/pull/2286 + + [Spark-3490] Disable SparkUI for tests + Andrew Or + 2014-09-11 17:18:46 -0700 + Commit: 2ffc798, github.com/apache/spark/pull/2363 + + [SPARK-2140] Updating heap memory calculation for YARN stable and alpha. + Chris Cope + 2014-09-11 08:13:07 -0500 + Commit: 06fb2d0, github.com/apache/spark/pull/2253 + + HOTFIX: Changing color on doc menu + Patrick Wendell + 2014-09-10 22:14:55 -0700 + Commit: e51ce9a + + [SPARK-1919] Fix Windows spark-shell --jars + Andrew Or + 2014-09-02 10:47:05 -0700 + Commit: 359cd59, github.com/apache/spark/pull/2211 + + [SPARK-3061] Fix Maven build under Windows + Josh Rosen , Josh Rosen , Josh Rosen + 2014-09-02 10:45:14 -0700 + Commit: 23fd3e8, github.com/apache/spark/pull/2165 + + [SPARK-3345] Do correct parameters for ShuffleFileGroup + Liang-Chi Hsieh + 2014-09-03 17:04:53 -0700 + Commit: e5f77ae, github.com/apache/spark/pull/2235 + + [SPARK-3193]output errer info when Process exit code is not zero in test suite + scwf + 2014-09-09 11:57:01 -0700 + Commit: 2426268, github.com/apache/spark/pull/2108 + + SPARK-2425 Don't kill a still-running Application because of some misbehaving Executors + Mark Hamstra + 2014-09-08 20:51:56 -0700 + Commit: e884805, github.com/apache/spark/pull/1360 + + [SQL] Minor edits to sql programming guide. + Henry Cook + 2014-09-08 14:56:37 -0700 + Commit: 7a236dc, github.com/apache/spark/pull/2316 + + [SPARK-938][doc] Add OpenStack Swift support + Reynold Xin , Gil Vernik + 2014-09-07 20:56:04 -0700 + Commit: 8c6306a, github.com/apache/spark/pull/is + + Fixed typos in make-distribution.sh + Cheng Lian + 2014-09-07 20:38:32 -0700 + Commit: e45bfa8, github.com/apache/spark/pull/2121 + + [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle. + Reynold Xin + 2014-09-07 18:42:24 -0700 + Commit: d555c2e, github.com/apache/spark/pull/2281 + + [SQL] Update SQL Programming Guide + Michael Armbrust , Yin Huai + 2014-09-07 21:34:46 -0400 + Commit: 65dae63, github.com/apache/spark/pull/2258 + + [SPARK-3394] [SQL] Fix crash in TakeOrdered when limit is 0 + Eric Liang + 2014-09-07 17:57:59 -0700 + Commit: c5d8d82, github.com/apache/spark/pull/2264 + + [SPARK-2419][Streaming][Docs] More updates to the streaming programming guide + Tathagata Das , Chris Fregly + 2014-09-06 14:46:43 -0700 + Commit: ce4053c, github.com/apache/spark/pull/2307 + + SPARK-3211 .take() is OOM-prone with empty partitions + Andrew Ash + 2014-09-05 18:52:05 -0700 + Commit: 28ce67b, github.com/apache/spark/pull/2117 + + [Docs] fix minor MLlib case typo + Nicholas Chammas + 2014-09-04 23:37:06 -0700 + Commit: 6b128be, github.com/apache/spark/pull/2278 + + [SPARK-3401][PySpark] Wrong usage of tee command in python/run-tests + Kousuke Saruta + 2014-09-04 10:29:11 -0700 + Commit: dbf8120, github.com/apache/spark/pull/2272 + + [HOTFIX] [SPARK-3400] Revert 9b225ac "fix GraphX EdgeRDD zipPartitions" + Ankur Dave + 2014-09-03 23:49:47 -0700 + Commit: 8c40ab5, github.com/apache/spark/pull/2271 + + [SPARK-3372] [MLlib] MLlib doesn't pass maven build / checkstyle due to multi-byte character contained in Gradient.scala + Kousuke Saruta + 2014-09-03 20:47:00 -0700 + Commit: f41c45a, github.com/apache/spark/pull/2248 + + [SPARK-2419][Streaming][Docs] Updates to the streaming programming guide + Tathagata Das , Jacek Laskowski + 2014-09-03 17:38:01 -0700 + Commit: 3111501, github.com/apache/spark/pull/2254 + + Release 1.1.0 [SPARK-3320][SQL] Made batched in-memory column buffer building work for SchemaRDDs with empty partitions diff --git a/assembly/pom.xml b/assembly/pom.xml index 9853cdfb66872..2780af37bd680 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml @@ -267,6 +267,16 @@ ${deb.install.path}/lib + + ${basedir}/../lib_managed/jars + directory + + perm + ${deb.user} + ${deb.user} + ${deb.install.path}/lib + + ${basedir}/src/deb/RELEASE file diff --git a/bagel/pom.xml b/bagel/pom.xml index 536fe516407b8..0c808acd592f8 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 89c3b165c8972..d362e5572ad11 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ee11547070638..cea5cf2022b7b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1334,7 +1334,7 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { - private[spark] val SPARK_VERSION = "1.1.0" + private[spark] val SPARK_VERSION = "1.1.1" private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b9501c3e4bbdb..3113d4a3149fc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -139,6 +139,11 @@ private[spark] class BlockManager( private val broadcastCleaner = new MetadataCleaner( MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) + // Field related to peer block managers that are necessary for block replication + @volatile private var cachedPeers: Seq[BlockManagerId] = _ + private val peerFetchLock = new Object + private var lastPeerFetchTime = 0L + initialize() /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay @@ -822,28 +827,111 @@ private[spark] class BlockManager( } /** - * Replicate block to another node. + * Get peer block managers in the system. + */ + private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + peerFetchLock.synchronized { + val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds + val timeout = System.currentTimeMillis - lastPeerFetchTime > cachedPeersTtl + if (cachedPeers == null || forceFetch || timeout) { + cachedPeers = master.getPeers(blockManagerId).sortBy(_.hashCode) + lastPeerFetchTime = System.currentTimeMillis + logDebug("Fetched peers from master: " + cachedPeers.mkString("[", ",", "]")) + } + cachedPeers + } + } + + /** + * Replicate block to another node. Not that this is a blocking call that returns after + * the block has been replicated. */ - @volatile var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { + val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) + val numPeersToReplicateTo = level.replication - 1 + val peersForReplication = new ArrayBuffer[BlockManagerId] + val peersReplicatedTo = new ArrayBuffer[BlockManagerId] + val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId] val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) - if (cachedPeers == null) { - cachedPeers = master.getPeers(blockManagerId, level.replication - 1) + val startTime = System.currentTimeMillis + val random = new Random(blockId.hashCode) + + var replicationFailed = false + var failures = 0 + var done = false + + // Get cached list of peers + peersForReplication ++= getPeers(forceFetch = false) + + // Get a random peer. Note that this selection of a peer is deterministic on the block id. + // So assuming the list of peers does not change and no replication failures, + // if there are multiple attempts in the same node to replicate the same block, + // the same set of peers will be selected. + def getRandomPeer(): Option[BlockManagerId] = { + // If replication had failed, then force update the cached list of peers and remove the peers + // that have been already used + if (replicationFailed) { + peersForReplication.clear() + peersForReplication ++= getPeers(forceFetch = true) + peersForReplication --= peersReplicatedTo + peersForReplication --= peersFailedToReplicateTo + } + if (!peersForReplication.isEmpty) { + Some(peersForReplication(random.nextInt(peersForReplication.size))) + } else { + None + } } - for (peer: BlockManagerId <- cachedPeers) { - val start = System.nanoTime - data.rewind() - logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + - s"To node: $peer") - val putBlock = PutBlock(blockId, data, tLevel) - val cmId = new ConnectionManagerId(peer.host, peer.port) - val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) - if (!syncPutBlockSuccess) { - logError(s"Failed to call syncPutBlock to $peer") + + // One by one choose a random peer and try uploading the block to it + // If replication fails (e.g., target peer is down), force the list of cached peers + // to be re-fetched from driver and then pick another random peer for replication. Also + // temporarily black list the peer for which replication failed. + // + // This selection of a peer and replication is continued in a loop until one of the + // following 3 conditions is fulfilled: + // (i) specified number of peers have been replicated to + // (ii) too many failures in replicating to peers + // (iii) no peer left to replicate to + // + while (!done) { + getRandomPeer() match { + case Some(peer) => + val onePeerStartTime = System.currentTimeMillis + data.rewind() + logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") + val putBlock = PutBlock(blockId, data, tLevel) + val cmId = new ConnectionManagerId(peer.host, peer.port) + val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) + if (syncPutBlockSuccess) { + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %d ms" + .format((System.currentTimeMillis - onePeerStartTime))) + peersReplicatedTo += peer + peersForReplication -= peer + replicationFailed = false + if (peersReplicatedTo.size == numPeersToReplicateTo) { + done = true // specified number of peers have been replicated to + } + } else { + logWarning(s"Failed to replicate $blockId to $peer, failure #$failures") + failures += 1 + replicationFailed = true + peersFailedToReplicateTo += peer + if (failures > maxReplicationFailures) { // too many failures in replicating to peers + done = true + } + } + case None => // no peer left to replicate to + done = true } - logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." - .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) + } + val timeTakeMs = (System.currentTimeMillis - startTime) + logTrace(s"Replicating $blockId of ${data.limit()} bytes to " + + s"${peersReplicatedTo.size} peer(s) took $timeTakeMs ms") + if (peersReplicatedTo.size < numPeersToReplicateTo) { + logWarning(s"Block $blockId replicated to only " + + s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index ffd2a4d3ef7a5..fb9305ac33943 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -62,7 +62,9 @@ class BlockManagerId private ( def nettyPort: Int = nettyPort_ - override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { + def isDriver: Boolean = (executorId == "") + + override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e67b3dc5ce02e..c2365ca643200 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -80,13 +80,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log } /** Get ids of other nodes in the cluster from the driver */ - def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { - val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) - if (result.length != numPeers) { - throw new SparkException( - "Error getting peers, only got " + result.size + " instead of " + numPeers) - } - result + def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index dc80148e137ed..ad1b68b2ac120 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -83,8 +83,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetLocationsMultipleBlockIds(blockIds) => sender ! getLocationsMultipleBlockIds(blockIds) - case GetPeers(blockManagerId, size) => - sender ! getPeers(blockManagerId, size) + case GetPeers(blockManagerId) => + sender ! getPeers(blockManagerId) case GetMemoryStatus => sender ! memoryStatus @@ -173,11 +173,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus * from the executors, but not from the driver. */ private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { - // TODO: Consolidate usages of import context.dispatcher val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) val requiredBlockManagers = blockManagerInfo.values.filter { info => - removeFromDriver || info.blockManagerId.executorId != "" + removeFromDriver || !info.blockManagerId.isDriver } Future.sequence( requiredBlockManagers.map { bm => @@ -213,7 +212,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val minSeenTime = now - slaveTimeout val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime && info.blockManagerId.executorId != "") { + if (info.lastSeenMs < minSeenTime && !info.blockManagerId.isDriver) { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId @@ -233,7 +232,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus */ private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerId.executorId == "" && !isLocal + blockManagerId.isDriver && !isLocal } else { blockManagerInfo(blockManagerId).updateLastSeenMs() true @@ -355,7 +354,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.executorId == "" && !isLocal) { + if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. sender ! true @@ -403,16 +402,14 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockIds.map(blockId => getLocations(blockId)) } - private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = { - val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - - val selfIndex = peers.indexOf(blockManagerId) - if (selfIndex == -1) { - throw new SparkException("Self index for " + blockManagerId + " not found") + /** Get the list of the peers of the given block manager */ + private def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + val blockManagerIds = blockManagerInfo.keySet + if (blockManagerIds.contains(blockManagerId)) { + blockManagerIds.filterNot { _.isDriver }.filterNot { _ == blockManagerId }.toSeq + } else { + Seq.empty } - - // Note that this logic will select the same node multiple times if there aren't enough peers - Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 03ba898f038be..291ddfcc113ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -90,7 +90,7 @@ private[spark] object BlockManagerMessages { case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster - case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6d3bef007e6e4..a9371c8208d13 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1528,7 +1528,7 @@ private[spark] object Utils extends Logging { def isBindCollision(exception: Throwable): Boolean = { exception match { case e: BindException => - if (e.getMessage != null && e.getMessage.contains("Address already in use")) { + if (e.getMessage != null) { return true } isBindCollision(e.getCause) diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 978a6ded80829..acaf321de52fb 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -132,7 +132,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === 1) statuses.head match { case (bm, status) => - assert(bm.executorId === "", "Block should only be on the driver") + assert(bm.isDriver, "Block should only be on the driver") assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) assert(status.memSize > 0, "Block should be in memory store on the driver") assert(status.diskSize === 0, "Block should not be in disk store on the driver") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala new file mode 100644 index 0000000000000..a7f7c76e486d0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -0,0 +1,410 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.{implicitConversions, postfixOps} + +import akka.actor.{ActorSystem, Props} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage.StorageLevel._ +import org.apache.spark.util.AkkaUtils + +/** Testsuite that tests block replication in BlockManager */ +class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAndAfter { + + private val conf = new SparkConf(false) + var actorSystem: ActorSystem = null + var master: BlockManagerMaster = null + val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) + + // List of block manager created during an unit test, so that all of the them can be stopped + // after the unit test. + val allStores = new ArrayBuffer[BlockManager] + + // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test + conf.set("spark.kryoserializer.buffer.mb", "1") + val serializer = new KryoSerializer(conf) + + // Implicitly convert strings to BlockIds for test clarity. + implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, + mapOutputTracker, shuffleManager) + allStores += store + store + } + + before { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) + this.actorSystem = actorSystem + + conf.set("spark.authenticate", "false") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + + // to make a replication attempt to inactive store fail fast + conf.set("spark.core.connection.ack.wait.timeout", "1") + // to make cached peers refresh frequently + conf.set("spark.storage.cachedPeersTtl", "10") + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) + allStores.clear() + } + + after { + allStores.foreach { _.stop() } + allStores.clear() + actorSystem.shutdown() + actorSystem.awaitTermination() + actorSystem = null + master = null + } + + + test("get peers with addition and removal of block managers") { + val numStores = 4 + val stores = (1 to numStores - 1).map { i => makeBlockManager(1000, s"store$i") } + val storeIds = stores.map { _.blockManagerId }.toSet + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId }) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId }) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId }) + + // Add driver store and test whether it is filtered out + val driverStore = makeBlockManager(1000, "") + assert(master.getPeers(stores(0).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(1).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(2).blockManagerId).forall(!_.isDriver)) + + // Add a new store and test whether get peers returns it + val newStore = makeBlockManager(1000, s"store$numStores") + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(newStore.blockManagerId).toSet === storeIds) + + // Remove a store and test whether get peers returns it + val storeIdToRemove = stores(0).blockManagerId + master.removeExecutor(storeIdToRemove.executorId) + assert(!master.getPeers(stores(1).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(stores(2).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(newStore.blockManagerId).contains(storeIdToRemove)) + + // Test whether asking for peers of a unregistered block manager id returns empty list + assert(master.getPeers(stores(0).blockManagerId).isEmpty) + assert(master.getPeers(BlockManagerId("", "", 1, 0)).isEmpty) + } + + test("block replication - 2x replication") { + testReplication(2, + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER_2) + ) + } + + test("block replication - 3x replication") { + // Generate storage levels with 3x replication + val storageLevels = { + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK, MEMORY_AND_DISK_SER).map { + level => StorageLevel( + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 3) + } + } + testReplication(3, storageLevels) + } + + test("block replication - mixed between 1x to 5x") { + // Generate storage levels with varying replication + val storageLevels = Seq( + MEMORY_ONLY, + MEMORY_ONLY_SER_2, + StorageLevel(true, false, false, false, 3), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, true, false, false, 5), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, false, false, false, 3), + MEMORY_ONLY_SER_2, + MEMORY_ONLY + ) + testReplication(5, storageLevels) + } + + test("block replication - 2x replication without peers") { + intercept[org.scalatest.exceptions.TestFailedException] { + testReplication(1, + Seq(StorageLevel.MEMORY_AND_DISK_2, StorageLevel(true, false, false, false, 3))) + } + } + + test("block replication - deterministic node selection") { + val blockSize = 1000 + val storeSize = 10000 + val stores = (1 to 5).map { + i => makeBlockManager(storeSize, s"store$i") + } + val storageLevel2x = StorageLevel.MEMORY_AND_DISK_2 + val storageLevel3x = StorageLevel(true, true, false, true, 3) + val storageLevel4x = StorageLevel(true, true, false, true, 4) + + def putBlockAndGetLocations(blockId: String, level: StorageLevel): Set[BlockManagerId] = { + stores.head.putSingle(blockId, new Array[Byte](blockSize), level) + val locations = master.getLocations(blockId).sortBy { _.executorId }.toSet + stores.foreach { _.removeBlock(blockId) } + master.removeBlock(blockId) + locations + } + + // Test if two attempts to 2x replication returns same set of locations + val a1Locs = putBlockAndGetLocations("a1", storageLevel2x) + assert(putBlockAndGetLocations("a1", storageLevel2x) === a1Locs, + "Inserting a 2x replicated block second time gave different locations from the first") + + // Test if two attempts to 3x replication returns same set of locations + val a2Locs3x = putBlockAndGetLocations("a2", storageLevel3x) + assert(putBlockAndGetLocations("a2", storageLevel3x) === a2Locs3x, + "Inserting a 3x replicated block second time gave different locations from the first") + + // Test if 2x replication of a2 returns a strict subset of the locations of 3x replication + val a2Locs2x = putBlockAndGetLocations("a2", storageLevel2x) + assert( + a2Locs2x.subsetOf(a2Locs3x), + "Inserting a with 2x replication gave locations that are not a subset of locations" + + s" with 3x replication [3x: ${a2Locs3x.mkString(",")}; 2x: ${a2Locs2x.mkString(",")}" + ) + + // Test if 4x replication of a2 returns a strict superset of the locations of 3x replication + val a2Locs4x = putBlockAndGetLocations("a2", storageLevel4x) + assert( + a2Locs3x.subsetOf(a2Locs4x), + "Inserting a with 4x replication gave locations that are not a superset of locations " + + s"with 3x replication [3x: ${a2Locs3x.mkString(",")}; 4x: ${a2Locs4x.mkString(",")}" + ) + + // Test if 3x replication of two different blocks gives two different sets of locations + val a3Locs3x = putBlockAndGetLocations("a3", storageLevel3x) + assert(a3Locs3x !== a2Locs3x, "Two blocks gave same locations with 3x replication") + } + + test("block replication - replication failures") { + /* + Create a system of three block managers / stores. One of them (say, failableStore) + cannot receive blocks. So attempts to use that as replication target fails. + + +-----------/fails/-----------> failableStore + | + normalStore + | + +-----------/works/-----------> anotherNormalStore + + We are first going to add a normal block manager (i.e. normalStore) and the failable block + manager (i.e. failableStore), and test whether 2x replication fails to create two + copies of a block. Then we are going to add another normal block manager + (i.e., anotherNormalStore), and test that now 2x replication works as the + new store will be used for replication. + */ + + // Add a normal block manager + val store = makeBlockManager(10000, "store") + + // Insert a block with 2x replication and return the number of copies of the block + def replicateAndGetNumCopies(blockId: String): Int = { + store.putSingle(blockId, new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK_2) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // Add a failable block manager with a mock transfer service that does not + // allow receiving of blocks. So attempts to use it as a replication target will fail. + val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, + 10000, conf, securityMgr, mapOutputTracker, shuffleManager) + failableStore.connectionManager.stop() // To disable any transfer to this store + allStores += failableStore // so that this gets stopped after test + assert(master.getPeers(store.blockManagerId).toSet === Set(failableStore.blockManagerId)) + + // Test that 2x replication fails by creating only one copy of the block + assert(replicateAndGetNumCopies("a1") === 1) + + // Add another normal block manager and test that 2x replication works + makeBlockManager(10000, "anotherStore") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a2") === 2) + } + } + + test("block replication - addition and deletion of block managers") { + val blockSize = 1000 + val storeSize = 10000 + val initialStores = (1 to 2).map { i => makeBlockManager(storeSize, s"store$i") } + + // Insert a block with given replication factor and return the number of copies of the block\ + def replicateAndGetNumCopies(blockId: String, replicationFactor: Int): Int = { + val storageLevel = StorageLevel(true, true, false, true, replicationFactor) + initialStores.head.putSingle(blockId, new Array[Byte](blockSize), storageLevel) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // 2x replication should work, 3x replication should only replicate 2x + assert(replicateAndGetNumCopies("a1", 2) === 2) + assert(replicateAndGetNumCopies("a2", 3) === 2) + + // Add another store, 3x replication should work now, 4x replication should only replicate 3x + val newStore1 = makeBlockManager(storeSize, s"newstore1") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a3", 3) === 3) + } + assert(replicateAndGetNumCopies("a4", 4) === 3) + + // Add another store, 4x replication should work now + val newStore2 = makeBlockManager(storeSize, s"newstore2") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a5", 4) === 4) + } + + // Remove all but the 1st store, 2x replication should fail + (initialStores.tail ++ Seq(newStore1, newStore2)).foreach { + store => + master.removeExecutor(store.blockManagerId.executorId) + store.stop() + } + assert(replicateAndGetNumCopies("a6", 2) === 1) + + // Add new stores, 3x replication should work + val newStores = (3 to 5).map { + i => makeBlockManager(storeSize, s"newstore$i") + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a7", 3) === 3) + } + } + + /** + * Test replication of blocks with different storage levels (various combinations of + * memory, disk & serialization). For each storage level, this function tests every store + * whether the block is present and also tests the master whether its knowledge of blocks + * is correct. Then it also drops the block from memory of each store (using LRU) and + * again checks whether the master's knowledge gets updated. + */ + private def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) { + import org.apache.spark.storage.StorageLevel._ + + assert(maxReplication > 1, + s"Cannot test replication factor $maxReplication") + + // storage levels to test with the given replication factor + + val storeSize = 10000 + val blockSize = 1000 + + // As many stores as the replication factor + val stores = (1 to maxReplication).map { + i => makeBlockManager(storeSize, s"store$i") + } + + storageLevels.foreach { storageLevel => + // Put the block into one of the stores + val blockId = new TestBlockId( + "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase) + stores(0).putSingle(blockId, new Array[Byte](blockSize), storageLevel) + + // Assert that master know two locations for the block + val blockLocations = master.getLocations(blockId).map(_.executorId).toSet + assert(blockLocations.size === storageLevel.replication, + s"master did not have ${storageLevel.replication} locations for $blockId, " + blockLocations) + + // Test state of the stores that contain the block + stores.filter { + testStore => blockLocations.contains(testStore.blockManagerId.executorId) + }.foreach { testStore => + val testStoreName = testStore.blockManagerId.executorId + assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), + s"master does not have status for ${blockId.name} in $testStoreName") + + val blockStatus = master.getBlockStatus(blockId)(testStore.blockManagerId) + + // Assert that block status in the master for this store has expected storage level + assert( + blockStatus.storageLevel.useDisk === storageLevel.useDisk && + blockStatus.storageLevel.useMemory === storageLevel.useMemory && + blockStatus.storageLevel.useOffHeap === storageLevel.useOffHeap && + blockStatus.storageLevel.deserialized === storageLevel.deserialized, + s"master does not know correct storage level for ${blockId.name} in $testStoreName") + + // Assert that the block status in the master for this store has correct memory usage info + assert(!blockStatus.storageLevel.useMemory || blockStatus.memSize >= blockSize, + s"master does not know size of ${blockId.name} stored in memory of $testStoreName") + + + // If the block is supposed to be in memory, then drop the copy of the block in + // this store test whether master is updated with zero memory usage this store + if (storageLevel.useMemory) { + // Force the block to be dropped by adding a number of dummy blocks + (1 to 10).foreach { + i => + testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + } + (1 to 10).foreach { + i => testStore.removeBlock(s"dummy-block-$i") + } + + val newBlockStatusOption = master.getBlockStatus(blockId).get(testStore.blockManagerId) + + // Assert that the block status in the master either does not exist (block removed + // from every store) or has zero memory usage for this store + assert( + newBlockStatusOption.isEmpty || newBlockStatusOption.get.memSize === 0, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in memory of $testStoreName" + ) + } + + // If the block is supposed to be in disk (after dropping or otherwise, then + // test whether master has correct disk usage for this store + if (storageLevel.useDisk) { + assert(master.getBlockStatus(blockId)(testStore.blockManagerId).diskSize >= blockSize, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in disk of $testStoreName" + ) + } + } + master.removeBlock(blockId) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index f32ce6f9fcc7f..48c45bfe6b3e9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -189,7 +189,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = makeBlockManager(2000, "exec1") store2 = makeBlockManager(2000, "exec2") - val peers = master.getPeers(store.blockManagerId, 1) + val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") @@ -448,7 +448,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) - System.out.println(list2DiskGet) // We don't know the exact size of the data on disk, but it should certainly be > 0. assert(list2DiskGet.get.inputMetrics.bytesRead > 0) assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 70d423ba8a04d..a530e0be183b0 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -22,6 +22,8 @@ import scala.util.Random import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} +import java.text.DecimalFormatSymbols +import java.util.Locale import com.google.common.base.Charsets import com.google.common.io.Files @@ -101,14 +103,16 @@ class UtilsSuite extends FunSuite { val hour = minute * 60 def str = Utils.msDurationToString(_) + val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator() + assert(str(123) === "123 ms") - assert(str(second) === "1.0 s") - assert(str(second + 462) === "1.5 s") - assert(str(hour) === "1.00 h") - assert(str(minute) === "1.0 m") - assert(str(minute + 4 * second + 34) === "1.1 m") - assert(str(10 * hour + minute + 4 * second) === "10.02 h") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") + assert(str(second) === "1" + sep + "0 s") + assert(str(second + 462) === "1" + sep + "5 s") + assert(str(hour) === "1" + sep + "00 h") + assert(str(minute) === "1" + sep + "0 m") + assert(str(minute + 4 * second + 34) === "1" + sep + "1 m") + assert(str(10 * hour + minute + 4 * second) === "10" + sep + "02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11" + sep + "00 h") } test("reading offset bytes of a file") { @@ -271,12 +275,11 @@ class UtilsSuite extends FunSuite { assert(!Utils.isBindCollision(new Exception)) assert(!Utils.isBindCollision(new Exception(new Exception))) assert(!Utils.isBindCollision(new Exception(new BindException))) - assert(!Utils.isBindCollision(new Exception(new BindException("Random message")))) // Positives - val be = new BindException("Address already in use") - val be1 = new Exception(new BindException("Address already in use")) - val be2 = new Exception(new Exception(new BindException("Address already in use"))) + val be = new BindException("Random Message") + val be1 = new Exception(new BindException("Random Message")) + val be2 = new Exception(new Exception(new BindException("Random Message"))) assert(Utils.isBindCollision(be)) assert(Utils.isBindCollision(be1)) assert(Utils.isBindCollision(be2)) diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 16ea1a71290dc..0b7069f6e116a 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -30,71 +30,84 @@ import time import urllib2 -# Fill in release details here: -RELEASE_URL = "http://people.apache.org/~pwendell/spark-1.0.0-rc1/" -RELEASE_KEY = "9E4FE3AF" -RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1006/" -RELEASE_VERSION = "1.0.0" +# Note: The following variables must be set before use! +RELEASE_URL = "http://people.apache.org/~andrewor14/spark-1.1.1-rc1/" +RELEASE_KEY = "XXXXXXXX" # Your 8-digit hex +RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1033" +RELEASE_VERSION = "1.1.1" SCALA_VERSION = "2.10.4" SCALA_BINARY_VERSION = "2.10" -# +# Do not set these LOG_FILE_NAME = "spark_audit_%s" % time.strftime("%h_%m_%Y_%I_%M_%S") LOG_FILE = open(LOG_FILE_NAME, 'w') WORK_DIR = "/tmp/audit_%s" % int(time.time()) MAVEN_CMD = "mvn" GPG_CMD = "gpg" +SBT_CMD = "sbt -Dsbt.log.noformat=true" -print "Starting tests, log output in %s. Test results printed below:" % LOG_FILE_NAME - -# Track failures +# Track failures to print them at the end failures = [] +# Log a message. Use sparingly because this flushes every write. +def log(msg): + LOG_FILE.write(msg + "\n") + LOG_FILE.flush() +def log_and_print(msg): + print msg + log(msg) + +# Prompt the user to delete the scratch directory used def clean_work_files(): - print "OK to delete scratch directory '%s'? (y/N): " % WORK_DIR - response = raw_input() + response = raw_input("OK to delete scratch directory '%s'? (y/N) " % WORK_DIR) if response == "y": shutil.rmtree(WORK_DIR) - print "Should I delete the log output file '%s'? (y/N): " % LOG_FILE_NAME - response = raw_input() - if response == "y": - os.unlink(LOG_FILE_NAME) - +# Run the given command and log its output to the log file def run_cmd(cmd, exit_on_failure=True): - print >> LOG_FILE, "Running command: %s" % cmd + log("Running command: %s" % cmd) ret = subprocess.call(cmd, shell=True, stdout=LOG_FILE, stderr=LOG_FILE) if ret != 0 and exit_on_failure: - print "Command failed: %s" % cmd + log_and_print("Command failed: %s" % cmd) clean_work_files() sys.exit(-1) return ret - def run_cmd_with_output(cmd): - print >> sys.stderr, "Running command: %s" % cmd + log_and_print("Running command: %s" % cmd) return subprocess.check_output(cmd, shell=True, stderr=LOG_FILE) +# Test if the given condition is successful +# If so, print the pass message; otherwise print the failure message +def test(cond, msg): + return passed(msg) if cond else failed(msg) -def test(bool, str): - if bool: - return passed(str) - failed(str) - - -def passed(str): - print "[PASSED] %s" % str - - -def failed(str): - failures.append(str) - print "[**FAILED**] %s" % str +def passed(msg): + log_and_print("[PASSED] %s" % msg) +def failed(msg): + failures.append(msg) + log_and_print("[**FAILED**] %s" % msg) def get_url(url): return urllib2.urlopen(url).read() +# If the path exists, prompt the user to delete it +# If the resource is not deleted, abort +def ensure_path_not_present(path): + full_path = os.path.expanduser(path) + if os.path.exists(full_path): + print "Found %s locally." % full_path + response = raw_input("This can interfere with testing published artifacts. OK to delete? (y/N) ") + if response == "y": + shutil.rmtree(full_path) + else: + print "Abort." + sys.exit(-1) + +log_and_print("|-------- Starting Spark audit tests for release %s --------|" % RELEASE_VERSION) +log_and_print("Log output can be found in %s" % LOG_FILE_NAME) original_dir = os.getcwd() @@ -114,37 +127,36 @@ def get_url(url): cache_ivy_spark = "~/.ivy2/cache/org.apache.spark" local_maven_kafka = "~/.m2/repository/org/apache/kafka" local_maven_kafka = "~/.m2/repository/org/apache/spark" - - -def ensure_path_not_present(x): - if os.path.exists(os.path.expanduser(x)): - print "Please remove %s, it can interfere with testing published artifacts." % x - sys.exit(-1) - map(ensure_path_not_present, [local_ivy_spark, cache_ivy_spark, local_maven_kafka]) # SBT build tests +log_and_print("==== Building SBT modules ====") os.chdir("blank_sbt_build") os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: + log("==== Building module %s in SBT ====" % module) os.environ["SPARK_MODULE"] = module - ret = run_cmd("sbt clean update", exit_on_failure=False) - test(ret == 0, "sbt build against '%s' module" % module) + ret = run_cmd("%s clean update" % SBT_CMD, exit_on_failure=False) + test(ret == 0, "SBT build against '%s' module" % module) os.chdir(original_dir) # SBT application tests +log_and_print("==== Building SBT applications ====") for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive", "sbt_app_kinesis"]: + log("==== Building application %s in SBT ====" % app) os.chdir(app) - ret = run_cmd("sbt clean run", exit_on_failure=False) - test(ret == 0, "sbt application (%s)" % app) + ret = run_cmd("%s clean run" % SBT_CMD, exit_on_failure=False) + test(ret == 0, "SBT application (%s)" % app) os.chdir(original_dir) # Maven build tests os.chdir("blank_maven_build") +log_and_print("==== Building Maven modules ====") for module in modules: + log("==== Building module %s in maven ====" % module) cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' '-Dspark.module="%s" clean compile' % (MAVEN_CMD, RELEASE_REPOSITORY, RELEASE_VERSION, module)) @@ -152,6 +164,8 @@ def ensure_path_not_present(x): test(ret == 0, "maven build against '%s' module" % module) os.chdir(original_dir) +# Maven application tests +log_and_print("==== Building Maven applications ====") os.chdir("maven_app_core") mvn_exec_cmd = ('%s --update-snapshots -Dspark.release.repository="%s" -Dspark.version="%s" ' '-Dscala.binary.version="%s" clean compile ' @@ -172,15 +186,14 @@ def ensure_path_not_present(x): artifact_regex = r = re.compile("") artifacts = r.findall(index_page) +# Verify artifact integrity for artifact in artifacts: - print "==== Verifying download integrity for artifact: %s ====" % artifact + log_and_print("==== Verifying download integrity for artifact: %s ====" % artifact) artifact_url = "%s/%s" % (RELEASE_URL, artifact) - run_cmd("wget %s" % artifact_url) - key_file = "%s.asc" % artifact + run_cmd("wget %s" % artifact_url) run_cmd("wget %s/%s" % (RELEASE_URL, key_file)) - run_cmd("wget %s%s" % (artifact_url, ".sha")) # Verify signature @@ -208,31 +221,17 @@ def ensure_path_not_present(x): os.chdir(WORK_DIR) -for artifact in artifacts: - print "==== Verifying build and tests for artifact: %s ====" % artifact - os.chdir(os.path.join(WORK_DIR, dir_name)) - - os.environ["MAVEN_OPTS"] = "-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - # Verify build - print "==> Running build" - run_cmd("sbt assembly") - passed("sbt build successful") - run_cmd("%s package -DskipTests" % MAVEN_CMD) - passed("Maven build successful") - - # Verify tests - print "==> Performing unit tests" - run_cmd("%s test" % MAVEN_CMD) - passed("Tests successful") - os.chdir(WORK_DIR) - -clean_work_files() - +# Report result +log_and_print("\n") if len(failures) == 0: - print "ALL TESTS PASSED" + log_and_print("*** ALL TESTS PASSED ***") else: - print "SOME TESTS DID NOT PASS" + log_and_print("XXXXX SOME TESTS DID NOT PASS XXXXX") for f in failures: - print f - + log_and_print(" %s" % f) os.chdir(original_dir) + +# Clean up +clean_work_files() + +log_and_print("|-------- Spark release audit complete --------|") diff --git a/dev/audit-release/blank_sbt_build/build.sbt b/dev/audit-release/blank_sbt_build/build.sbt index 696c7f651837c..62815542e5bd9 100644 --- a/dev/audit-release/blank_sbt_build/build.sbt +++ b/dev/audit-release/blank_sbt_build/build.sbt @@ -19,10 +19,12 @@ name := "Spark Release Auditor" version := "1.0" -scalaVersion := "2.9.3" +scalaVersion := System.getenv.get("SCALA_VERSION") libraryDependencies += "org.apache.spark" % System.getenv.get("SPARK_MODULE") % System.getenv.get("SPARK_VERSION") resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Eclipse Paho Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", + "Maven Repository" at "http://repo1.maven.org/maven2/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_hive/build.sbt b/dev/audit-release/sbt_app_hive/build.sbt index a0d4f25da5842..c8824f2b15e55 100644 --- a/dev/audit-release/sbt_app_hive/build.sbt +++ b/dev/audit-release/sbt_app_hive/build.sbt @@ -25,4 +25,5 @@ libraryDependencies += "org.apache.spark" %% "spark-hive" % System.getenv.get("S resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Maven Repository" at "http://repo1.maven.org/maven2/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml b/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml deleted file mode 100644 index 93b835813d535..0000000000000 --- a/dev/audit-release/sbt_app_hive/src/main/resources/hive-site.xml +++ /dev/null @@ -1,213 +0,0 @@ - - - - - - - - - - - - - - - - - - build.dir - ${user.dir}/build - - - - build.dir.hive - ${build.dir}/hive - - - - hadoop.tmp.dir - ${build.dir.hive}/test/hadoop-${user.name} - A base for other temporary directories. - - - - - - hive.exec.scratchdir - ${build.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${build.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - javax.jdo.option.ConnectionURL - - jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - ${build.dir}/test/data/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${build.dir}/test/logs - - - - - test.src.dir - ${build.dir}/src/test - - - - - - - hive.jar.path - ${build.dir.hive}/ql/hive-exec-${version}.jar - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${build.dir}/tmp - Location of the structured hive logs - - - - - - hive.task.progress - false - Track progress of a task - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 281e8d4de6d71..91d8789bc9fd7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -27,6 +27,7 @@ # Would be nice to add: # - Send output to stderr and have useful logging in stdout +# Note: The following variables must be set before use! GIT_USERNAME=${GIT_USERNAME:-pwendell} GIT_PASSWORD=${GIT_PASSWORD:-XXX} GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} @@ -101,7 +102,7 @@ make_binary_release() { cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME - ./make-distribution.sh --name $NAME --tgz $FLAGS + ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . rm -rf spark-$RELEASE_VERSION-bin-$NAME diff --git a/dev/create-release/generate-changelist.py b/dev/create-release/generate-changelist.py index 916ec90f05844..e40205058cb43 100755 --- a/dev/create-release/generate-changelist.py +++ b/dev/create-release/generate-changelist.py @@ -30,14 +30,17 @@ import time import traceback -SPARK_HOME = os.environ["SPARK_HOME"] -NEW_RELEASE_VERSION = "1.1.0" -PREV_RELEASE_GIT_TAG = "v1.0.0" +# Note: These variables must be updated before use! +NEW_RELEASE_VERSION = "1.1.1" +PREV_RELEASE_GIT_TAG = "1.1.0" +# Do not set these +SPARK_HOME = os.environ["SPARK_HOME"] CHANGELIST = "CHANGES.txt" OLD_CHANGELIST = "%s.old" % (CHANGELIST) NEW_CHANGELIST = "%s.new" % (CHANGELIST) TMP_CHANGELIST = "%s.tmp" % (CHANGELIST) +PREV_RELEASE_GIT_TAG = "v" + PREV_RELEASE_GIT_TAG # date before first PR in TLP Spark repo SPARK_REPO_CHANGE_DATE1 = time.strptime("2014-02-26", "%Y-%m-%d") diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 943f06b114cb9..63d28b8966d70 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -4,7 +4,7 @@ title: Running Spark on YARN --- Support for running on [YARN (Hadoop -NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) +NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html) was added to Spark in version 0.6.0, and improved in subsequent releases. # Preparations diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 2e36345365766..374d3a6a949e9 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -84,7 +84,7 @@ def parse_args(): "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") parser.add_option( - "-v", "--spark-version", default="1.1.0", + "-v", "--spark-version", default="1.1.1", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option( "--spark-git-repo", @@ -197,7 +197,7 @@ def is_active(instance): def get_spark_shark_version(opts): spark_shark_map = { "0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", - "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0" + "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0", "1.1.1": "1.1.1" } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: diff --git a/examples/pom.xml b/examples/pom.xml index 66eda645f5dc3..14cb6ef1245cf 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index 45bcedebb4117..bcbea3b6579eb 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -35,8 +35,9 @@ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * * Usage: JavaNetworkWordCount - * and describe the TCP server that Spark Streaming would connect to receive data. + * and describe the TCP server that Spark Streaming would connect to receive data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` @@ -56,7 +57,7 @@ public static void main(String[] args) { // Create the context with a 1 second batch size SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); - JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java new file mode 100644 index 0000000000000..0b8a1c38d87db --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.Arrays; +import java.util.regex.Pattern; + +import scala.Tuple2; +import com.google.common.collect.Lists; +import com.google.common.io.Files; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.Time; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaStreamingContextFactory; + +/** + * Counts words in text encoded with UTF8 received from the network every second. + * + * Usage: JavaRecoverableNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive + * data. directory to HDFS-compatible file system which checkpoint data + * file to which the word counts will be appended + * + * and must be absolute paths + * + * To run this on your local machine, you need to first run a Netcat server + * + * `$ nc -lk 9999` + * + * and run the example as + * + * `$ ./bin/run-example org.apache.spark.examples.streaming.JavaRecoverableNetworkWordCount \ + * localhost 9999 ~/checkpoint/ ~/out` + * + * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create + * a new StreamingContext (will print "Creating new context" to the console). Otherwise, if + * checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from + * the checkpoint data. + * + * Refer to the online documentation for more details. + */ +public final class JavaRecoverableNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + private static JavaStreamingContext createContext(String ip, + int port, + String checkpointDirectory, + String outputPath) { + + // If you do not see this printed, that means the StreamingContext has been loaded + // from the new checkpoint + System.out.println("Creating new context"); + final File outputFile = new File(outputPath); + if (outputFile.exists()) { + outputFile.delete(); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaRecoverableNetworkWordCount"); + // Create the context with a 1 second batch size + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); + ssc.checkpoint(checkpointDirectory); + + // Create a socket stream on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + JavaReceiverInputDStream lines = ssc.socketTextStream(ip, port); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + wordCounts.foreachRDD(new Function2, Time, Void>() { + @Override + public Void call(JavaPairRDD rdd, Time time) throws IOException { + String counts = "Counts at time " + time + " " + rdd.collect(); + System.out.println(counts); + System.out.println("Appending to " + outputFile.getAbsolutePath()); + Files.append(counts + "\n", outputFile, Charset.defaultCharset()); + return null; + } + }); + + return ssc; + } + + public static void main(String[] args) { + if (args.length != 4) { + System.err.println("You arguments were " + Arrays.asList(args)); + System.err.println( + "Usage: JavaRecoverableNetworkWordCount \n" + + " . and describe the TCP server that Spark\n" + + " Streaming would connect to receive data. directory to\n" + + " HDFS-compatible file system which checkpoint data file to which\n" + + " the word counts will be appended\n" + + "\n" + + "In local mode, should be 'local[n]' with n > 1\n" + + "Both and must be absolute paths"); + System.exit(1); + } + + final String ip = args[0]; + final int port = Integer.parseInt(args[1]); + final String checkpointDirectory = args[2]; + final String outputPath = args[3]; + JavaStreamingContextFactory factory = new JavaStreamingContextFactory() { + @Override + public JavaStreamingContext create() { + return createContext(ip, port, checkpointDirectory, outputPath); + } + }; + JavaStreamingContext ssc = JavaStreamingContext.getOrCreate(checkpointDirectory, factory); + ssc.start(); + ssc.awaitTermination(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 6af3a0f33efc2..19427e629f76d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -31,15 +31,13 @@ import org.apache.spark.util.IntParam /** * Counts words in text encoded with UTF8 received from the network every second. * - * Usage: NetworkWordCount + * Usage: RecoverableNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive * data. directory to HDFS-compatible file system which checkpoint data * file to which the word counts will be appended * - * In local mode, should be 'local[n]' with n > 1 * and must be absolute paths * - * * To run this on your local machine, you need to first run a Netcat server * * `$ nc -lk 9999` @@ -54,22 +52,11 @@ import org.apache.spark.util.IntParam * checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from * the checkpoint data. * - * To run this example in a local standalone cluster with automatic driver recovery, - * - * `$ bin/spark-class org.apache.spark.deploy.Client -s launch \ - * \ - * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ - * localhost 9999 ~/checkpoint ~/out` - * - * would typically be - * /examples/target/scala-XX/spark-examples....jar - * * Refer to the online documentation for more details. */ - object RecoverableNetworkWordCount { - def createContext(ip: String, port: Int, outputPath: String) = { + def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) = { // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint @@ -79,6 +66,7 @@ object RecoverableNetworkWordCount { val sparkConf = new SparkConf().setAppName("RecoverableNetworkWordCount") // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) + ssc.checkpoint(checkpointDirectory) // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') @@ -114,7 +102,7 @@ object RecoverableNetworkWordCount { val Array(ip, IntParam(port), checkpointDirectory, outputPath) = args val ssc = StreamingContext.getOrCreate(checkpointDirectory, () => { - createContext(ip, port, outputPath) + createContext(ip, port, outputPath, checkpointDirectory) }) ssc.start() ssc.awaitTermination() diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index cba215131d0df..51c13593dd870 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..4411d6e20c52a --- /dev/null +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 75a6668c6210b..a49e4aee13aae 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -157,6 +157,7 @@ class SparkSinkSuite extends FunSuite { channelContext.put("transactionCapacity", 1000.toString) channelContext.put("keep-alive", 0.toString) channelContext.putAll(overrides) + channel.setName(scala.util.Random.nextString(10)) channel.configure(channelContext) val sink = new SparkSink() diff --git a/external/flume/pom.xml b/external/flume/pom.xml index ccbeaeb3f8ea4..f8d3d01e77c1c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 21972471101e4..f06400fcf7742 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 8a347cf6a8227..69e2f69287169 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 03f939d011d63..628b7b202bb29 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 2a92f2cc7d910..bd592d29cf5b5 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 8ce7b94fee1be..e1f0d7fadaf97 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0 + 1.1.1 ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 387de26acb517..6590f12b7b5a8 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-csd-1-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 7c3c2a2729cb0..e1cdc7afbb2c5 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 26041cf0138ab..d8c1676a22af6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 7d845c44365dd..f23eb5b96d6d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -104,21 +104,19 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val deltaMean: BDV[Double] = currMean - other.currMean var i = 0 while (i < n) { - // merge mean together - if (other.currMean(i) != 0.0) { + if (nnz(i) + other.nnz(i) != 0.0) { + // merge mean together currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / (nnz(i) + other.nnz(i)) - } - // merge m2n together - if (nnz(i) + other.nnz(i) != 0.0) { + // merge m2n together currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / (nnz(i) + other.nnz(i)) - } - if (currMax(i) < other.currMax(i)) { - currMax(i) = other.currMax(i) - } - if (currMin(i) > other.currMin(i)) { - currMin(i) = other.currMin(i) + if (currMax(i) < other.currMax(i)) { + currMax(i) = other.currMax(i) + } + if (currMin(i) > other.currMin(i)) { + currMin(i) = other.currMin(i) + } } i += 1 } diff --git a/pom.xml b/pom.xml index 80f23cfd844af..daba907769a0d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -135,7 +135,6 @@ 0.3.6 3.0.0 1.7.6 - 0.7.1 1.8.3 1.1.0 @@ -622,7 +621,6 @@ org.apache.avro avro-mapred ${avro.version} - ${avro.mapred.classifier} io.netty @@ -1096,7 +1094,6 @@ 2.2.0 2.5.0 - hadoop2 @@ -1106,7 +1103,6 @@ 2.3.0 2.5.0 0.9.0 - hadoop2 @@ -1116,7 +1112,6 @@ 2.4.0 2.5.0 0.9.0 - hadoop2 diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 357626edc9622..82f76de31afc1 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -40,6 +40,7 @@ # These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. DEFAULT_CONFIGS = { + "spark.serializer": "org.apache.spark.serializer.KryoSerializer", "spark.serializer.objectStreamReset": 100, "spark.rdd.compress": True, } diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index e863fc249ec36..d4c06c05663da 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -60,6 +60,18 @@ def predictAll(self, usersProducts): class ALS(object): + """Alternating Least Squares matrix factorization. + + SPARK-3990: In Spark 1.1.x, we use Kryo serialization by default in + PySpark. ALS does not work under this default setting. You can switch + back to the default Java serialization by setting: + + spark.serializer=org.apache.spark.serializer.JavaSerializer + + Please go to http://spark.apache.org/docs/latest/configuration.html + for instructions on how to configure Spark. + """ + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 8a9777a708b33..0ee431cf6749c 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -47,7 +47,7 @@ ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /__ / .__/\_,_/_/ /_/\_\ version 1.1.0 + /__ / .__/\_,_/_/ /_/\_\ version 1.1.1 /_/ """) print("Using Python version %s (%s, %s)" % ( diff --git a/repl/pom.xml b/repl/pom.xml index 6370b0a8d3d4a..1a4c422672cc7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 74c58eb49d07f..d834261fbb91d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -26,7 +26,7 @@ trait SparkILoopInit { ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 1.1.0 + /___/ .__/\_,_/_/ /_/\_\ version 1.1.1 /_/ """) import Properties._ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7e574458dbcd1..72112f083d605 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 6e010e4977f61..3ae8e18825e2b 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a75af94d29303..4889fea24af6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -75,6 +75,11 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith("spark.sql") => setConf(key, value) + case _ => + } + /** * :: DeveloperApi :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 031b695169cea..3429fbad024c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -48,43 +48,35 @@ case class SetCommand( extends LeafNode with Command with Logging { override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { - // Set value for key k. - case (Some(k), Some(v)) => - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + // Configures the deprecated "mapred.reduce.tasks" property. + case (Some(SQLConf.Deprecated.MAPRED_REDUCE_TASKS), Some(v)) => + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") - } else { - context.setConf(k, v) - Array(s"$k=$v") - } - - // Query the value bound to key k. - case (Some(k), _) => - // TODO (lian) This is just a workaround to make the Simba ODBC driver work. - // Should remove this once we get the ODBC driver updated. - if (k == "-v") { - val hiveJars = Seq( - "hive-exec-0.12.0.jar", - "hive-service-0.12.0.jar", - "hive-common-0.12.0.jar", - "hive-hwi-0.12.0.jar", - "hive-0.12.0.jar").mkString(":") - - Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getConf(k, "")}") - } - - // Query all key-value pairs that are set in the SQLConf of the context. - case (None, None) => - context.getAllConfs.map { case (k, v) => - s"$k=$v" - }.toSeq + context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) + Seq(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + + // Configures a single property. + case (Some(k), Some(v)) => + context.setConf(k, v) + Seq(s"$k=$v") + + // Queries all key-value pairs that are set in the SQLConf of the context. Notice that different + // from Hive, here "SET -v" is an alias of "SET". (In Hive, "SET" returns all changed properties + // while "SET -v" returns all properties.) + case (Some("-v") | None, None) => + context.getAllConfs.map { case (k, v) => s"$k=$v" }.toSeq + + // Queries the deprecated "mapred.reduce.tasks" property. + case (Some(SQLConf.Deprecated.MAPRED_REDUCE_TASKS), None) => + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") + Seq(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}") + + // Queries a single property. + case (Some(k), None) => + Seq(s"$k=${context.getConf(k, "")}") case _ => throw new IllegalArgumentException() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index f2389f8f0591e..4fdfc2ba1ba6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -17,9 +17,18 @@ package org.apache.spark.sql.test +import org.apache.spark.sql.{SQLConf, SQLContext} import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext /** A SQLContext that can be used for local testing. */ object TestSQLContext - extends SQLContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends SQLContext( + new SparkContext( + "local[2]", + "TestSQLContext", + new SparkConf().set("spark.sql.testkey", "true"))) { + + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 584f71b3c13d5..60701f0e154f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql +import org.scalatest.FunSuiteLike + import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ -class SQLConfSuite extends QueryTest { +class SQLConfSuite extends QueryTest with FunSuiteLike { val testKey = "test.key.0" val testVal = "test.val.0" + test("propagate from spark conf") { + // We create a new context here to avoid order dependence with other tests that might call + // clear(). + val newContext = new SQLContext(TestSQLContext.sparkContext) + assert(newContext.getConf("spark.sql.testkey", "false") == "true") + } + test("programmatic ways of basic setting and getting") { clear() assert(getAllConfs.size === 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 42923b6a288d9..c6b790a4b6a23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -63,8 +63,7 @@ case class AllDataTypes( doubleField: Double, shortField: Short, byteField: Byte, - booleanField: Boolean, - binaryField: Array[Byte]) + booleanField: Boolean) case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -75,13 +74,14 @@ case class AllDataTypesWithNonPrimitiveType( shortField: Short, byteField: Byte, booleanField: Boolean, - binaryField: Array[Byte], array: Seq[Int], arrayContainsNull: Seq[Option[Int]], map: Map[Int, Long], mapValueContainsNull: Map[Int, Option[Long]], data: Data) +case class BinaryData(binaryData: Array[Byte]) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -117,26 +117,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - TestSQLContext.sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, - (0 to x).map(_.toByte).toArray)) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - assert(result(i).getShort(5) === i.toShort) - assert(result(i).getByte(6) === i.toByte) - assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === (0 to i).map(_.toByte).toArray) - } + val data = sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } - test("Treat binary as string") { + test("read/write binary data") { + // Since equality for Array[Byte] is broken we test this separately. + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + sparkContext.parallelize(BinaryData("test".getBytes("utf8")) :: Nil).saveAsParquetFile(tempDir) + parquetFile(tempDir) + .map(r => new String(r(0).asInstanceOf[Array[Byte]], "utf8")) + .collect().toSeq == Seq("test") + } + + ignore("Treat binary as string") { val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString // Create the test file. @@ -151,37 +151,16 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA StructField("c2", BinaryType, false) :: Nil) val schemaRDD1 = applySchema(rowRDD, schema) schemaRDD1.saveAsParquetFile(path) - val resultWithBinary = parquetFile(path).collect - range.foreach { - i => - assert(resultWithBinary(i).getInt(0) === i) - assert(resultWithBinary(i)(1) === s"val_$i".getBytes) - } - - TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") - // This ParquetRelation always use Parquet types to derive output. - val parquetRelation = new ParquetRelation( - path.toString, - Some(TestSQLContext.sparkContext.hadoopConfiguration), - TestSQLContext) { - override val output = - ParquetTypesConverter.convertToAttributes( - ParquetTypesConverter.readMetaData(new Path(path), conf).getFileMetaData.getSchema, - TestSQLContext.isParquetBinaryAsString) - } - val schemaRDD = new SchemaRDD(TestSQLContext, parquetRelation) - val resultWithString = schemaRDD.collect - range.foreach { - i => - assert(resultWithString(i).getInt(0) === i) - assert(resultWithString(i)(1) === s"val_$i") - } + checkAnswer( + parquetFile(path).select('c1, 'c2.cast(StringType)), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) - schemaRDD.registerTempTable("tmp") + setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") + parquetFile(path).printSchema() checkAnswer( - sql("SELECT c1, c2 FROM tmp WHERE c2 = 'val_5' OR c2 = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) + parquetFile(path), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) + // Set it back. TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) @@ -284,34 +263,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - TestSQLContext.sparkContext.parallelize(range) + val data = sparkContext.parallelize(range) .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, - (0 to x).map(_.toByte).toArray, (0 until x), (0 until x).map(Option(_).filter(_ % 3 == 0)), (0 until x).map(i => i -> i.toLong).toMap, (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), Data((0 until x), Nested(x, s"$x")))) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - assert(result(i).getShort(5) === i.toShort) - assert(result(i).getByte(6) === i.toByte) - assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === (0 to i).map(_.toByte).toArray) - assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) - assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) - assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) - assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) - } + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } test("self-join parquet files") { @@ -408,23 +372,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } - test("Saving case class RDD table to file and reading it back in") { - val file = getTempFilePath("parquet") - val path = file.toString - val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) - .map(i => TestRDDEntry(i, s"val_$i")) - rdd.saveAsParquetFile(path) - val readFile = parquetFile(path) - readFile.registerTempTable("tmpx") - val rdd_copy = sql("SELECT * FROM tmpx").collect() - val rdd_orig = rdd.collect() - for(i <- 0 to 99) { - assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") - assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value error in line $i") - } - Utils.deleteRecursively(file) - } - test("Read a parquet file instead of a directory") { val file = getTempFilePath("parquet") val path = file.toString @@ -457,32 +404,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() assert(rdd_copy1.size === 100) - assert(rdd_copy1(0).apply(0) === 1) - assert(rdd_copy1(0).apply(1) === "val_1") - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! + sql("INSERT INTO dest SELECT * FROM source") - val rdd_copy2 = sql("SELECT * FROM dest").collect() + val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) assert(rdd_copy2.size === 200) - assert(rdd_copy2(0).apply(0) === 1) - assert(rdd_copy2(0).apply(1) === "val_1") - assert(rdd_copy2(99).apply(0) === 100) - assert(rdd_copy2(99).apply(1) === "val_100") - assert(rdd_copy2(100).apply(0) === 1) - assert(rdd_copy2(100).apply(1) === "val_1") Utils.deleteRecursively(dirname) } test("Insert (appending) to same table via Scala API") { - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) - for(i <- (0 to 14)) { - assert(double_rdd(i) === double_rdd(i+15), s"error: lines $i and ${i+15} to not match") - } + // let's restore the original test data Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6339d791d980..c94b7439c928f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index cadf7aaf42157..161f8c6199b08 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ - import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} @@ -38,24 +35,12 @@ private[hive] object HiveThriftServer2 extends Logging { def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - if (!optionsProcessor.process(args)) { System.exit(-1) } - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logDebug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - logInfo("Starting SparkContext") SparkSQLEnv.init() - SessionState.start(ss) Runtime.getRuntime.addShutdownHook( new Thread() { @@ -67,7 +52,7 @@ private[hive] object HiveThriftServer2 extends Logging { try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) + server.init(SparkSQLEnv.hiveContext.hiveconf) server.start() logInfo("HiveThriftServer2 started") } catch { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 42cbf363b274f..94ec9978af85f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -17,18 +17,18 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ - import java.io.IOException import java.util.{List => JList} import javax.security.auth.login.LoginException +import scala.collection.JavaConversions._ + import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory -import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.cli._ import org.apache.hive.service.{AbstractService, Service, ServiceException} import org.apache.spark.sql.hive.HiveContext @@ -57,6 +57,15 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) initCompositeService(hiveConf) } + + override def getInfo(sessionHandle: SessionHandle, getInfoType: GetInfoType): GetInfoValue = { + getInfoType match { + case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") + case GetInfoType.CLI_DBMS_NAME => new GetInfoValue("Spark SQL") + case GetInfoType.CLI_DBMS_VER => new GetInfoValue(hiveContext.sparkContext.version) + case _ => super.getInfo(sessionHandle, getInfoType) + } + } } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 582264eb59f83..e07402c56c5b9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive.thriftserver -import org.apache.hadoop.hive.ql.session.SessionState +import scala.collection.JavaConversions._ -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.Logging +import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -33,14 +32,18 @@ private[hive] object SparkSQLEnv extends Logging { def init() { if (hiveContext == null) { - sparkContext = new SparkContext(new SparkConf() - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + val sparkConf = new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") + .set("spark.sql.hive.version", "0.12.0") + sparkContext = new SparkContext(sparkConf) sparkContext.addSparkListener(new StatsReportListener()) + hiveContext = new HiveContext(sparkContext) - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() - @transient override lazy val hiveconf = sessionState.getConf + if (log.isDebugEnabled) { + hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => + logDebug(s"HiveConf var: $k=$v") + } } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index bd3f68d92d8c7..37bd97a680328 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -113,7 +113,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) case ShortType => - to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal))) + to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) case TimestampType => to.addColumnValue( ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) @@ -145,7 +145,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(null)) case ShortType => - to.addColumnValue(ColumnValue.intValue(null)) + to.addColumnValue(ColumnValue.shortValue(null)) case TimestampType => to.addColumnValue(ColumnValue.timestampValue(null)) case BinaryType | _: ArrayType | _: StructType | _: MapType => @@ -199,6 +199,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => + setState(OperationState.ERROR) logError("Error executing query:",e) throw new HiveSQLException(e.toString) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 3475c2c9db080..e8ffbc5b954d4 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -18,15 +18,13 @@ package org.apache.spark.sql.hive.thriftserver +import java.io._ + import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ -import scala.concurrent.{Await, Future, Promise} +import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} -import java.io._ -import java.util.concurrent.atomic.AtomicInteger - import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -53,17 +51,19 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin.split("\\s+").toSeq ++ extraArgs } - // AtomicInteger is needed because stderr and stdout of the forked process are handled in - // different threads. - val next = new AtomicInteger(0) + var next = 0 val foundAllExpectedAnswers = Promise.apply[Unit]() val queryStream = new ByteArrayInputStream(queries.mkString("\n").getBytes) val buffer = new ArrayBuffer[String]() + val lock = new Object - def captureOutput(source: String)(line: String) { + def captureOutput(source: String)(line: String): Unit = lock.synchronized { buffer += s"$source> $line" - if (line.contains(expectedAnswers(next.get()))) { - if (next.incrementAndGet() == expectedAnswers.size) { + // If we haven't found all expected answers and another expected answer comes up... + if (next < expectedAnswers.size && line.startsWith(expectedAnswers(next))) { + next += 1 + // If all expected answers have been found... + if (next == expectedAnswers.size) { foundAllExpectedAnswers.trySuccess(()) } } @@ -73,11 +73,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val process = (Process(command) #< queryStream).run( ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - Future { - val exitValue = process.exitValue() - logInfo(s"Spark SQL CLI process exit value: $exitValue") - } - try { Await.result(foundAllExpectedAnswers.future, timeout) } catch { case cause: Throwable => @@ -88,14 +83,15 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { |======================= |Spark SQL CLI command line: ${command.mkString(" ")} | - |Executed query ${next.get()} "${queries(next.get())}", - |But failed to capture expected output "${expectedAnswers(next.get())}" within $timeout. + |Executed query $next "${queries(next)}", + |But failed to capture expected output "${expectedAnswers(next)}" within $timeout. | |${buffer.mkString("\n")} |=========================== |End CliSuite failure output |=========================== """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() @@ -107,7 +103,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - runCliWithin(1.minute)( + runCliWithin(3.minute)( "CREATE TABLE hive_test(key INT, val STRING);" -> "OK", "SHOW TABLES;" @@ -118,7 +114,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { -> "Time taken: ", "SELECT COUNT(*) FROM hive_test;" -> "5", - "DROP TABLE hive_test" + "DROP TABLE hive_test;" -> "Time taken: " ) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 38977ff162097..08b4cc1c42c31 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -17,32 +17,39 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration._ -import scala.concurrent.{Await, Future, Promise} -import scala.sys.process.{Process, ProcessLogger} - import java.io.File import java.net.ServerSocket import java.sql.{DriverManager, Statement} import java.util.concurrent.TimeoutException +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.concurrent.{Await, Promise} +import scala.sys.process.{Process, ProcessLogger} + import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.GetInfoType +import org.apache.hive.service.cli.thrift.TCLIService.Client +import org.apache.hive.service.cli.thrift._ +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.TSocket import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.sql.catalyst.util.getTempFilePath /** * Tests for the HiveThriftServer2 using JDBC. + * + * NOTE: SPARK_PREPEND_CLASSES is explicitly disabled in this test suite. Assembly jar must be + * rebuilt after changing HiveThriftServer2 related code. */ class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) - private val listeningHost = "localhost" - private val listeningPort = { + def randomListeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. val socket = new ServerSocket(0) @@ -51,61 +58,91 @@ class HiveThriftServer2Suite extends FunSuite with Logging { port } - private val warehousePath = getTempFilePath("warehouse") - private val metastorePath = getTempFilePath("metastore") - private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + def withJdbcStatement(serverStartTimeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + val port = randomListeningPort + + startThriftServer(port, serverStartTimeout) { + val jdbcUri = s"jdbc:hive2://${"localhost"}:$port/" + val user = System.getProperty("user.name") + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() - def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { - val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + try { + f(statement) + } finally { + statement.close() + connection.close() + } + } + } + + def withCLIServiceClient( + serverStartTimeout: FiniteDuration = 1.minute)( + f: ThriftCLIServiceClient => Unit) { + val port = randomListeningPort + + startThriftServer(port) { + // Transport creation logics below mimics HiveConnection.createBinaryTransport + val rawTransport = new TSocket("localhost", port) + val user = System.getProperty("user.name") + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val protocol = new TBinaryProtocol(transport) + val client = new ThriftCLIServiceClient(new Client(protocol)) + + transport.open() + + try { + f(client) + } finally { + transport.close() + } + } + } + def startThriftServer( + port: Int, + serverStartTimeout: FiniteDuration = 1.minute)( + f: => Unit) { + val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" val command = - s"""$serverScript + s"""$startScript | --master local | --hiveconf hive.root.logger=INFO,console | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$listeningHost - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=${"localhost"} + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port """.stripMargin.split("\\s+").toSeq - val serverStarted = Promise[Unit]() + val serverRunning = Promise[Unit]() val buffer = new ArrayBuffer[String]() + val lock = new Object - def captureOutput(source: String)(line: String) { + def captureOutput(source: String)(line: String): Unit = lock.synchronized { buffer += s"$source> $line" if (line.contains("ThriftBinaryCLIService listening on")) { - serverStarted.success(()) + serverRunning.success(()) } } - val process = Process(command).run( - ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - - Future { - val exitValue = process.exitValue() - logInfo(s"Spark SQL Thrift server process exit value: $exitValue") - } + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + val env = Seq("SPARK_TESTING" -> "0") - val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" - val user = System.getProperty("user.name") + val process = Process(command, None, env: _*).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) try { - Await.result(serverStarted.future, timeout) - - val connection = DriverManager.getConnection(jdbcUri, user, "") - val statement = connection.createStatement() - - try { - f(statement) - } finally { - statement.close() - connection.close() - } + Await.result(serverRunning.future, serverStartTimeout) + f } catch { case cause: Exception => cause match { case _: TimeoutException => - logError(s"Failed to start Hive Thrift server within $timeout", cause) + logError(s"Failed to start Hive Thrift server within $serverStartTimeout", cause) case _ => } logError( @@ -114,14 +151,15 @@ class HiveThriftServer2Suite extends FunSuite with Logging { |HiveThriftServer2Suite failure output |===================================== |HiveThriftServer2 command line: ${command.mkString(" ")} - |JDBC URI: $jdbcUri - |User: $user + |Binding port: $port + |System user: ${System.getProperty("user.name")} | |${buffer.mkString("\n")} |========================================= |End HiveThriftServer2Suite failure output |========================================= """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() @@ -130,14 +168,16 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } test("Test JDBC query execution") { - startThriftServerWithin() { statement => + withJdbcStatement() { statement => val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - val queries = Seq( - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test", - "CACHE TABLE test") + val queries = + s"""SET spark.sql.shuffle.partitions=3; + |CREATE TABLE test(key INT, val STRING); + |LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test; + |CACHE TABLE test; + """.stripMargin.split(";").map(_.trim).filter(_.nonEmpty) queries.foreach(statement.execute) @@ -150,7 +190,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } test("SPARK-3004 regression: result set containing NULL") { - startThriftServerWithin() { statement => + withJdbcStatement() { statement => val dataFilePath = Thread.currentThread().getContextClassLoader.getResource( "data/files/small_kv_with_null.txt") @@ -173,4 +213,31 @@ class HiveThriftServer2Suite extends FunSuite with Logging { assert(!resultSet.next()) } } + + test("GetInfo Thrift API") { + withCLIServiceClient() { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + assertResult("Spark SQL", "Wrong GetInfo(CLI_DBMS_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_NAME).getStringValue + } + + assertResult("Spark SQL", "Wrong GetInfo(CLI_SERVER_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_SERVER_NAME).getStringValue + } + + assertResult(SparkContext.SPARK_VERSION, "Spark version shouldn't be \"Unknown\"") { + client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_VER).getStringValue + } + } + } + + test("Checks Hive version") { + withJdbcStatement() { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === s"spark.sql.hive.version=0.12.0") + } + } } diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index cd5ad3541da3b..15ca0b97f2553 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../../pom.xml @@ -94,15 +94,6 @@ org.apache.avro avro - ${avro.version} - - - - org.apache.avro - avro-mapred - ${avro.version} - ${avro.mapred.classifier} org.scalatest diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index d9b2bc7348ad2..b44a94c6aed9a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -222,17 +222,29 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } /** - * SQLConf and HiveConf contracts: when the hive session is first initialized, params in - * HiveConf will get picked up by the SQLConf. Additionally, any properties set by - * set() or a SET command inside sql() will be set in the SQLConf *as well as* - * in the HiveConf. + * SQLConf and HiveConf contracts: + * + * 1. reuse existing started SessionState if any + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. */ - @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) - @transient protected[hive] lazy val sessionState = { - val ss = new SessionState(hiveconf) - setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. - ss - } + @transient protected[hive] lazy val (hiveconf, sessionState) = + Option(SessionState.get()) + .orElse { + val newState = new SessionState(new HiveConf(classOf[SessionState])) + // Only starts newly created `SessionState` instance. Any existing `SessionState` instance + // returned by `SessionState.get()` must be the most recently started one. + SessionState.start(newState) + Some(newState) + } + .map { state => + setConf(state.getConf.getAllProperties) + if (state.out == null) state.out = new PrintStream(outputBuffer, true, "UTF-8") + if (state.err == null) state.err = new PrintStream(outputBuffer, true, "UTF-8") + (state.getConf, state) + } + .get sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") @@ -290,6 +302,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { SessionState.start(sessionState) + // Makes sure the session represented by the `sessionState` field is activated. This implies + // Spark SQL Hive support uses a single `SessionState` for all Hive operations and breaks + // session isolation under multi-user scenarios (i.e. HiveThriftServer2). + // TODO Fix session isolation + if (SessionState.get() != sessionState) { + SessionState.start(sessionState) + } + proc match { case driver: Driver => driver.init() @@ -306,7 +326,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { driver.destroy() results case _ => - sessionState.out.println(tokens(0) + " " + cmd_1) + if (sessionState.out != null) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } Seq(proc.run(cmd_1).getResponseCode.toString) } } catch { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 71c70dc46a0e0..98287d102f90c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -660,7 +660,7 @@ private[hive] object HiveQl { def nodeToRelation(node: Node): LogicalPlan = node match { case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => - Subquery(alias, nodeToPlan(query)) + Subquery(cleanIdentifier(alias), nodeToPlan(query)) case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => val Token("TOK_SELECT", @@ -845,7 +845,7 @@ private[hive] object HiveQl { case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), alias)()) + Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) /* Hints are ignored */ case Token("TOK_HINTLIST", _) => None diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 8bb2216b7b4f4..094e58e9863c8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -35,12 +35,13 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{CacheCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ +import org.apache.spark.sql.SQLConf /* Implicit conversions */ import scala.collection.JavaConversions._ object TestHive - extends TestHiveContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends TestHiveContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) /** * A locally running test instance of Spark's Hive execution engine. @@ -90,6 +91,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index d2587431951b8..cdf984420782b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.execution import scala.util.Try -import org.apache.spark.sql.{SchemaRDD, Row} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -313,10 +312,10 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") test("case sensitivity: registered table") { - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: - TestData(2, "str2") :: Nil) + TestData(2, "str2") :: Nil).toSchemaRDD testData.registerTempTable("REGisteredTABle") assertResult(Array(Array(2, "str2"))) { @@ -327,7 +326,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.exists(_ == "== Physical Plan ==") + explanation.contains("== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { @@ -467,10 +466,10 @@ class HiveQuerySuite extends HiveComparisonTest { } // Describe a registered temporary table. - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: - TestData(1, "str2") :: Nil) + TestData(1, "str2") :: Nil).toSchemaRDD testData.registerTempTable("test_describe_commands2") assertResult( @@ -520,10 +519,15 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - + val KV = "([^=]+)=([^=]*)".r + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value + case Row(KV(key, value)) => key -> value + }.toSet clear() - // "set" itself returns all config variables currently specified in SQLConf. + // "SET" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... assert(sql("SET").collect().size == 0) @@ -532,46 +536,21 @@ class HiveQuerySuite extends HiveComparisonTest { } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) - } + assertResult(Set(testKey -> testVal))(collectResults(sql("SET"))) + assertResult(Set(testKey -> testVal))(collectResults(sql("SET -v"))) sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { sql(s"SET").collect().map(_.getString(0)) } - - // "set key" - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) - } - - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET -v")) } - // Assert that sql() should have the same effects as sql() by repeating the above using sql(). - clear() - assert(sql("SET").collect().size == 0) - - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) - } - - assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) - } - - sql(s"SET ${testKey + testKey}=${testVal + testVal}") - assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) - } - - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + // "SET key" + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } assertResult(Array(s"$nonexistentKey=")) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 635a9fb0d56cb..73d1186c68de5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -39,7 +39,7 @@ class SQLQuerySuite extends QueryTest { test("ordering not in agg") { checkAnswer( sql("SELECT key FROM src GROUP BY key, value ORDER BY value"), - sql(""" + sql( """ SELECT key FROM ( SELECT key, value @@ -47,4 +47,16 @@ class SQLQuerySuite extends QueryTest { GROUP BY key, value ORDER BY value) a""").collect().toSeq) } + + test("SPARK-3708 Backticks aren't handled correctly is aliases") { + checkAnswer( + sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"), + sql("SELECT `key` FROM src").collect().toSeq) + } + + test("SPARK-3834 Backticks not correctly handled in subquery aliases") { + checkAnswer( + sql("SELECT a.key FROM (SELECT key FROM src) `a`"), + sql("SELECT `key` FROM src").collect().toSeq) + } } diff --git a/streaming/pom.xml b/streaming/pom.xml index 76814c7b44880..0dd67a1205222 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 8152b7542ac57..55d6cf6a783ea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -120,14 +120,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas /** Generate one RDD from an array of files */ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = { - val fileRDDs = files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file)) - files.zip(fileRDDs).foreach { case (file, rdd) => { + val fileRDDs = files.map(file =>{ + val rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file) if (rdd.partitions.size == 0) { logError("File " + file + " has no data in it. Spark Streaming can only ingest " + "files that have been \"moved\" to the directory assigned to the file stream. " + "Refer to the streaming programming guide for more details.") } - }} + rdd + }) new UnionRDD(context.sparkContext, fileRDDs) } diff --git a/tools/pom.xml b/tools/pom.xml index faf9def55e2a7..a497ff176409c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-csd-4-SNAPSHOT + 1.1.1-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 8c951c74188ae..df08d0e48b01a 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,11 @@ org.apache.spark yarn-parent_2.10 +<<<<<<< HEAD 1.1.0-csd-1-SNAPSHOT +======= + 1.1.2-SNAPSHOT +>>>>>>> 685bdd2b7e584c84e7d39e40de2d5f30c5388cb5 ../pom.xml