aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-12936][SQL] Initial bloom filter implementationWenchen Fan2016-01-255-0/+602
| | | | | | | | | | | | | This PR adds an initial implementation of bloom filter in the newly added sketch module. The implementation is based on the [`BloomFilter` class in guava](https://code.google.com/p/guava-libraries/source/browse/guava/src/com/google/common/hash/BloomFilter.java). Some difference from the design doc: * expose `bitSize` instead of `sizeInBytes` to user. * always need the `expectedInsertions` parameter when create bloom filter. Author: Wenchen Fan <wenchen@databricks.com> Closes #10883 from cloud-fan/bloom-filter.
* [SPARK-12879] [SQL] improve the unsafe row writing frameworkWenchen Fan2016-01-257-78/+258
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | As we begin to use unsafe row writing framework(`BufferHolder` and `UnsafeRowWriter`) in more and more places(`UnsafeProjection`, `UnsafeRowParquetRecordReader`, `GenerateColumnAccessor`, etc.), we should add more doc to it and make it easier to use. This PR abstract the technique used in `UnsafeRowParquetRecordReader`: avoid unnecessary operatition as more as possible. For example, do not always point the row to the buffer at the end, we only need to update the size of row. If all fields are of primitive type, we can even save the row size updating. Then we can apply this technique to more places easily. a local benchmark shows `UnsafeProjection` is up to 1.7x faster after this PR: **old version** ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz unsafe projection: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- single long 2616.04 102.61 1.00 X single nullable long 3032.54 88.52 0.86 X primitive types 9121.05 29.43 0.29 X nullable primitive types 12410.60 21.63 0.21 X ``` **new version** ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz unsafe projection: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- single long 1533.34 175.07 1.00 X single nullable long 2306.73 116.37 0.66 X primitive types 8403.93 31.94 0.18 X nullable primitive types 12448.39 21.56 0.12 X ``` For single non-nullable long(the best case), we can have about 1.7x speed up. Even it's nullable, we can still have 1.3x speed up. For other cases, it's not such a boost as the saved operations only take a little proportion of the whole process. The benchmark code is included in this PR. Author: Wenchen Fan <wenchen@databricks.com> Closes #10809 from cloud-fan/unsafe-projection.
* [SPARK-12934][SQL] Count-min sketch serializationCheng Lian2016-01-254-19/+213
| | | | | | | | | | This PR adds serialization support for `CountMinSketch`. A version number is added to version the serialized binary format. Author: Cheng Lian <lian@databricks.com> Closes #10893 from liancheng/cms-serialization.
* [SPARK-12905][ML][PYSPARK] PCAModel return eigenvalues for PySparkYanbo Liang2016-01-252-0/+13
| | | | | | | | | | ```PCAModel``` can output ```explainedVariance``` at Python side. cc mengxr srowen Author: Yanbo Liang <ybliang8@gmail.com> Closes #10830 from yanboliang/spark-12905.
* [SPARK-12975][SQL] Throwing Exception when Bucketing Columns are part of ↵gatorsmile2016-01-253-3/+83
| | | | | | | | | | | | | | | | | | | | | | Partitioning Columns When users are using `partitionBy` and `bucketBy` at the same time, some bucketing columns might be part of partitioning columns. For example, ``` df.write .format(source) .partitionBy("i") .bucketBy(8, "i", "k") .saveAsTable("bucketed_table") ``` However, in the above case, adding column `i` into `bucketBy` is useless. It is just wasting extra CPU when reading or writing bucket tables. Thus, like Hive, we can issue an exception and let users do the change. Also added a test case for checking if the information of `sortBy` and `bucketBy` columns are correctly saved in the metastore table. Could you check if my understanding is correct? cloud-fan rxin marmbrus Thanks! Author: gatorsmile <gatorsmile@gmail.com> Closes #10891 from gatorsmile/commonKeysInPartitionByBucketBy.
* [SPARK-12901][SQL][HOT-FIX] Fix scala 2.11 compilation.Yin Huai2016-01-252-2/+2
|
* [SPARK-12902] [SQL] visualization for generated operatorsDavies Liu2016-01-259-32/+104
| | | | | | | | | | | | | | This PR brings back visualization for generated operators, they looks like: ![sql](https://cloud.githubusercontent.com/assets/40902/12460920/0dc7956a-bf6b-11e5-9c3f-8389f452526e.png) ![stage](https://cloud.githubusercontent.com/assets/40902/12460923/11806ac4-bf6b-11e5-9c72-e84a62c5ea93.png) Note: SQL metrics are not supported right now, because they are very slow, will be supported once we have batch mode. Author: Davies Liu <davies@databricks.com> Closes #10828 from davies/viz_codegen.
* [SPARK-12149][WEB UI] Executor UI improvement suggestions - Color UIAlex Bozarth2016-01-257-20/+103
| | | | | | | | | | | | | Added color coding to the Executors page for Active Tasks, Failed Tasks, Completed Tasks and Task Time. Active Tasks is shaded blue with it's range based on percentage of total cores used. Failed Tasks is shaded red ranging over the first 10% of total tasks failed Completed Tasks is shaded green ranging over 10% of total tasks including failed and active tasks, but only when there are active or failed tasks on that executor. Task Time is shaded red when GC Time goes over 10% of total time with it's range directly corresponding to the percent of total time. Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #10154 from ajbozarth/spark12149.
* Closes #10879Xiangrui Meng2016-01-250-0/+0
| | | | | | | | | Closes #9046 Closes #8532 Closes #10756 Closes #8960 Closes #10485 Closes #10467
* [SPARK-11965][ML][DOC] Update user guide for RFormula feature interactionsYanbo Liang2016-01-252-1/+40
| | | | | | | | Update user guide for RFormula feature interactions. Meanwhile we also update other new features such as supporting string label in Spark 1.6. Author: Yanbo Liang <ybliang8@gmail.com> Closes #10222 from yanboliang/spark-11965.
* [SPARK-12755][CORE] Stop the event logger before the DAG schedulerMichael Allman2016-01-251-6/+6
| | | | | | | | | | [SPARK-12755][CORE] Stop the event logger before the DAG scheduler to avoid a race condition where the standalone master attempts to build the app's history UI before the event log is stopped. This contribution is my original work, and I license this work to the Spark project under the project's open source license. Author: Michael Allman <michael@videoamp.com> Closes #10700 from mallman/stop_event_logger_first.
* [SPARK-12932][JAVA API] improved error message for java type inference failureAndy Grove2016-01-251-1/+2
| | | | | | Author: Andy Grove <andygrove73@gmail.com> Closes #10865 from andygrove/SPARK-12932.
* [SPARK-12901][SQL] Refactor options for JSON and CSV datasource (not case ↵hyukjinkwon2016-01-256-52/+40
| | | | | | | | | | | | | | | | | class and same format). https://issues.apache.org/jira/browse/SPARK-12901 This PR refactors the options in JSON and CSV datasources. In more details, 1. `JSONOptions` uses the same format as `CSVOptions`. 2. Not case classes. 3. `CSVRelation` that does not have to be serializable (it was `with Serializable` but I removed) Author: hyukjinkwon <gurwls223@gmail.com> Closes #10895 from HyukjinKwon/SPARK-12901.
* [SPARK-12624][PYSPARK] Checks row length when converting Java arrays to ↵Cheng Lian2016-01-242-1/+17
| | | | | | | | | | Python rows When actual row length doesn't conform to specified schema field length, we should give a better error message instead of throwing an unintuitive `ArrayOutOfBoundsException`. Author: Cheng Lian <lian@databricks.com> Closes #10886 from liancheng/spark-12624.
* [SPARK-12120][PYSPARK] Improve exception message when failing to init…Jeff Zhang2016-01-241-3/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | …ialize HiveContext in PySpark davies Mind to review ? This is the error message after this PR ``` 15/12/03 16:59:53 WARN ObjectStore: Failed to get database default, returning NoSuchObjectException /Users/jzhang/github/spark/python/pyspark/sql/context.py:689: UserWarning: You must build Spark with Hive. Export 'SPARK_HIVE=true' and run build/sbt assembly warnings.warn("You must build Spark with Hive. " Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/Users/jzhang/github/spark/python/pyspark/sql/context.py", line 663, in read return DataFrameReader(self) File "/Users/jzhang/github/spark/python/pyspark/sql/readwriter.py", line 56, in __init__ self._jreader = sqlContext._ssql_ctx.read() File "/Users/jzhang/github/spark/python/pyspark/sql/context.py", line 692, in _ssql_ctx raise e py4j.protocol.Py4JJavaError: An error occurred while calling None.org.apache.spark.sql.hive.HiveContext. : java.lang.RuntimeException: java.net.ConnectException: Call From jzhangMBPr.local/127.0.0.1 to 0.0.0.0:9000 failed on connection exception: java.net.ConnectException: Connection refused; For more details see: http://wiki.apache.org/hadoop/ConnectionRefused at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:522) at org.apache.spark.sql.hive.client.ClientWrapper.<init>(ClientWrapper.scala:194) at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:238) at org.apache.spark.sql.hive.HiveContext.executionHive$lzycompute(HiveContext.scala:218) at org.apache.spark.sql.hive.HiveContext.executionHive(HiveContext.scala:208) at org.apache.spark.sql.hive.HiveContext.functionRegistry$lzycompute(HiveContext.scala:462) at org.apache.spark.sql.hive.HiveContext.functionRegistry(HiveContext.scala:461) at org.apache.spark.sql.UDFRegistration.<init>(UDFRegistration.scala:40) at org.apache.spark.sql.SQLContext.<init>(SQLContext.scala:330) at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:90) at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:101) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:234) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:381) at py4j.Gateway.invoke(Gateway.java:214) at py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:79) at py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:68) at py4j.GatewayConnection.run(GatewayConnection.java:209) at java.lang.Thread.run(Thread.java:745) ``` Author: Jeff Zhang <zjffdu@apache.org> Closes #10126 from zjffdu/SPARK-12120.
* [SPARK-10498][TOOLS][BUILD] Add requirements.txt file for dev python toolsHolden Karau2016-01-241-0/+3
| | | | | | | | | | Minor since so few people use them, but it would probably be good to have a requirements file for our python release tools for easier setup (also version pinning). cc JoshRosen who looked at the original JIRA. Author: Holden Karau <holden@us.ibm.com> Closes #10871 from holdenk/SPARK-10498-add-requirements-file-for-dev-python-tools.
* [SPARK-12971] Fix Hive tests which fail in Hadoop-2.3 SBT buildJosh Rosen2016-01-242-4/+22
| | | | | | | | | | ErrorPositionSuite and one of the HiveComparisonTest tests have been consistently failing on the Hadoop 2.3 SBT build (but on no other builds). I believe that this is due to test isolation issues (e.g. tests sharing state via the sets of temporary tables that are registered to TestHive). This patch attempts to improve the isolation of these tests in order to address this issue. Author: Josh Rosen <joshrosen@databricks.com> Closes #10884 from JoshRosen/fix-failing-hadoop-2.3-hive-tests.
* [STREAMING][MINOR] Scaladoc + logsJacek Laskowski2016-01-234-6/+5
| | | | | | | | Found while doing code review Author: Jacek Laskowski <jacek@japila.pl> Closes #10878 from jaceklaskowski/streaming-scaladoc-logs-tiny-fixes.
* [SPARK-12904][SQL] Strength reduction for integral and decimal literal ↵Reynold Xin2016-01-236-139/+376
| | | | | | | | | | comparisons This pull request implements strength reduction for comparing integral expressions and decimal literals, which is more common now because we switch to parsing fractional literals as decimal types (rather than doubles). I added the rules to the existing DecimalPrecision rule with some refactoring to simplify the control flow. I also moved DecimalPrecision rule into its own file due to the growing size. Author: Reynold Xin <rxin@databricks.com> Closes #10882 from rxin/SPARK-12904-1.
* [SPARK-11137][STREAMING] Make StreamingContext.stop() exception-safejayadevanmurali2016-01-231-4/+12
| | | | | | | | Make StreamingContext.stop() exception-safe Author: jayadevanmurali <jayadevan.m@tcs.com> Closes #10807 from jayadevanmurali/branch-0.1-SPARK-11137.
* [SPARK-12760][DOCS] inaccurate description for difference between local vs ↵Sean Owen2016-01-231-4/+4
| | | | | | | | | | cluster mode in closure handling Clarify that modifying a driver local variable won't have the desired effect in cluster modes, and may or may not work as intended in local mode Author: Sean Owen <sowen@cloudera.com> Closes #10866 from srowen/SPARK-12760.
* [SPARK-12760][DOCS] invalid lambda expression in python example for …Mortada Mehyar2016-01-231-2/+5
| | | | | | | | | | | | | | | | | | | | | | | …local vs cluster srowen thanks for the PR at https://github.com/apache/spark/pull/10866! sorry it took me a while. This is related to https://github.com/apache/spark/pull/10866, basically the assignment in the lambda expression in the python example is actually invalid ``` In [1]: data = [1, 2, 3, 4, 5] In [2]: counter = 0 In [3]: rdd = sc.parallelize(data) In [4]: rdd.foreach(lambda x: counter += x) File "<ipython-input-4-fcb86c182bad>", line 1 rdd.foreach(lambda x: counter += x) ^ SyntaxError: invalid syntax ``` Author: Mortada Mehyar <mortada.mehyar@gmail.com> Closes #10867 from mortada/doc_python_fix.
* [SPARK-12859][STREAMING][WEB UI] Names of input streams with receivers don't ↵Alex Bozarth2016-01-231-1/+1
| | | | | | | | | | fit in Streaming page Added CSS style to force names of input streams with receivers to wrap Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #10873 from ajbozarth/spark12859.
* [SPARK-12933][SQL] Initial implementation of Count-Min sketchCheng Lian2016-01-239-12/+892
| | | | | | | | | | | | | | | | | | | This PR adds an initial implementation of count min sketch, contained in a new module spark-sketch under `common/sketch`. The implementation is based on the [`CountMinSketch` class in stream-lib][1]. As required by the [design doc][2], spark-sketch should have no external dependency. Two classes, `Murmur3_x86_32` and `Platform` are copied to spark-sketch from spark-unsafe for hashing facilities. They'll also be used in the upcoming bloom filter implementation. The following features will be added in future follow-up PRs: - Serialization support - DataFrame API integration [1]: https://github.com/addthis/stream-lib/blob/aac6b4d23a8686b000f80baa447e0922ecac3bcb/src/main/java/com/clearspring/analytics/stream/frequency/CountMinSketch.java [2]: https://issues.apache.org/jira/secure/attachment/12782378/BloomFilterandCount-MinSketchinSpark2.0.pdf Author: Cheng Lian <lian@databricks.com> Closes #10851 from liancheng/count-min-sketch.
* [SPARK-12872][SQL] Support to specify the option for compression codec for ↵hyukjinkwon2016-01-225-29/+96
| | | | | | | | | | | | | | | JSON datasource https://issues.apache.org/jira/browse/SPARK-12872 This PR makes the JSON datasource can compress output by option instead of manually setting Hadoop configurations. For reflecting codec by names, it is similar with https://github.com/apache/spark/pull/10805. As `CSVCompressionCodecs` can be shared with other datasources, it became a separate class to share as `CompressionCodecs`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #10858 from HyukjinKwon/SPARK-12872.
* [HOTFIX]Remove rpcEnv.awaitTermination to avoid dead-lock in some testShixiong Zhu2016-01-221-1/+0
| | | | Looks rpcEnv.awaitTermination may block some tests forever. Just remove it and investigate the tests.
* [SPARK-7997][CORE] Remove Akka from Spark Core and StreamingShixiong Zhu2016-01-2243-831/+123
| | | | | | | | | | | | - Remove Akka dependency from core. Note: the streaming-akka project still uses Akka. - Remove HttpFileServer - Remove Akka configs from SparkConf and SSLOptions - Rename `spark.akka.frameSize` to `spark.rpc.message.maxSize`. I think it's still worth to keep this config because using `DirectTaskResult` or `IndirectTaskResult` depends on it. - Update comments and docs Author: Shixiong Zhu <shixiong@databricks.com> Closes #10854 from zsxwing/remove-akka.
* [HOTFIX][BUILD][TEST-MAVEN] Remove duplicate dependencyShixiong Zhu2016-01-221-7/+4
| | | | | | Author: Shixiong Zhu <shixiong@databricks.com> Closes #10868 from zsxwing/hotfix-akka-pom.
* [SPARK-12629][SPARKR] Fixes for DataFrame saveAsTable methodNarine Kokhlikyan2016-01-223-9/+41
| | | | | | | | | | I've tried to solve some of the issues mentioned in: https://issues.apache.org/jira/browse/SPARK-12629 Please, let me know what do you think. Thanks! Author: Narine Kokhlikyan <narine.kokhlikyan@gmail.com> Closes #10580 from NarineK/sparkrSavaAsRable.
* [SPARK-12959][SQL] Writing Bucketed Data with Disabled Bucketing in SQLConfgatorsmile2016-01-223-6/+26
| | | | | | | | | | | | When users turn off bucketing in SQLConf, we should issue some messages to tell users these operations will be converted to normal way. Also added a test case for this scenario and fixed the helper function. Do you think this PR is helpful when using bucket tables? cloud-fan Thank you! Author: gatorsmile <gatorsmile@gmail.com> Closes #10870 from gatorsmile/bucketTableWritingTestcases.
* [SPARK-12960] [PYTHON] Some examples are missing support for python2Mark Grover2016-01-212-0/+3
| | | | | | | | Without importing the print_function, the lines later on like ```print("Usage: direct_kafka_wordcount.py <broker_list> <topic>", file=sys.stderr)``` fail when using python2.*. Import fixes that problem and doesn't break anything on python3 either. Author: Mark Grover <mark@apache.org> Closes #10872 from markgrover/python2_compat.
* [SPARK-12747][SQL] Use correct type name for Postgres JDBC's real arrayLiang-Chi Hsieh2016-01-213-3/+9
| | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-12747 Postgres JDBC driver uses "FLOAT4" or "FLOAT8" not "real". Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #10695 from viirya/fix-postgres-jdbc.
* [SPARK-12908][ML] Add warning message for LogisticRegression for potential ↵DB Tsai2016-01-211-0/+8
| | | | | | | | | | converge issue When all labels are the same, it's a dangerous ground for LogisticRegression without intercept to converge. GLMNET doesn't support this case, and will just exit. GLM can train, but will have a warning message saying the algorithm doesn't converge. Author: DB Tsai <dbt@netflix.com> Closes #10862 from dbtsai/add-tests.
* [SPARK-12534][DOC] update documentation to list command line equivalent to ↵felixcheung2016-01-213-6/+36
| | | | | | | | | | properties Several Spark properties equivalent to Spark submit command line options are missing. Author: felixcheung <felixcheung_m@hotmail.com> Closes #10491 from felixcheung/sparksubmitdoc.
* [SPARK-12204][SPARKR] Implement drop method for DataFrame in SparkR.Sun Rui2016-01-206-27/+101
| | | | | | Author: Sun Rui <rui.sun@intel.com> Closes #10201 from sun-rui/SPARK-12204.
* [SPARK-12910] Fixes : R version for installing sparkRShubhanshu Mishra2016-01-202-2/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Testing code: ``` $ ./install-dev.sh USING R_HOME = /usr/bin ERROR: this R is version 2.15.1, package 'SparkR' requires R >= 3.0 ``` Using the new argument: ``` $ ./install-dev.sh /content/username/SOFTWARE/R-3.2.3 USING R_HOME = /content/username/SOFTWARE/R-3.2.3/bin * installing *source* package ‘SparkR’ ... ** R ** inst ** preparing package for lazy loading Creating a new generic function for ‘colnames’ in package ‘SparkR’ Creating a new generic function for ‘colnames<-’ in package ‘SparkR’ Creating a new generic function for ‘cov’ in package ‘SparkR’ Creating a new generic function for ‘na.omit’ in package ‘SparkR’ Creating a new generic function for ‘filter’ in package ‘SparkR’ Creating a new generic function for ‘intersect’ in package ‘SparkR’ Creating a new generic function for ‘sample’ in package ‘SparkR’ Creating a new generic function for ‘transform’ in package ‘SparkR’ Creating a new generic function for ‘subset’ in package ‘SparkR’ Creating a new generic function for ‘summary’ in package ‘SparkR’ Creating a new generic function for ‘lag’ in package ‘SparkR’ Creating a new generic function for ‘rank’ in package ‘SparkR’ Creating a new generic function for ‘sd’ in package ‘SparkR’ Creating a new generic function for ‘var’ in package ‘SparkR’ Creating a new generic function for ‘predict’ in package ‘SparkR’ Creating a new generic function for ‘rbind’ in package ‘SparkR’ Creating a generic function for ‘lapply’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘Filter’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘alias’ from package ‘stats’ in package ‘SparkR’ Creating a generic function for ‘substr’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘%in%’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘mean’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘unique’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘nrow’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘ncol’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘head’ from package ‘utils’ in package ‘SparkR’ Creating a generic function for ‘factorial’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘atan2’ from package ‘base’ in package ‘SparkR’ Creating a generic function for ‘ifelse’ from package ‘base’ in package ‘SparkR’ ** help No man pages found in package ‘SparkR’ *** installing help indices ** building package indices ** testing if installed package can be loaded * DONE (SparkR) ``` Author: Shubhanshu Mishra <smishra8@illinois.edu> Closes #10836 from napsternxg/master.
* [SPARK-8968] [SQL] [HOT-FIX] Fix scala 2.11 build.Yin Huai2016-01-201-1/+1
|
* [SPARK-8968][SQL] external sort by the partition clomns when dynamic ↵wangfei2016-01-202-99/+166
| | | | | | | | | | | | | | | | | partitioning to optimize the memory overhead Now the hash based writer dynamic partitioning show the bad performance for big data and cause many small files and high GC. This patch we do external sort first so that each time we only need open one writer. before this patch: ![gc](https://cloud.githubusercontent.com/assets/7018048/9149788/edc48c6e-3dec-11e5-828c-9995b56e4d65.PNG) after this patch: ![gc-optimize-externalsort](https://cloud.githubusercontent.com/assets/7018048/9149794/60f80c9c-3ded-11e5-8a56-7ae18ddc7a2f.png) Author: wangfei <wangfei_hello@126.com> Author: scwf <wangfei1@huawei.com> Closes #7336 from scwf/dynamic-optimize-basedon-apachespark.
* [SPARK-12797] [SQL] Generated TungstenAggregate (without grouping keys)Davies Liu2016-01-205-12/+111
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | As discussed in #10786, the generated TungstenAggregate does not support imperative functions. For a query ``` sqlContext.range(10).filter("id > 1").groupBy().count() ``` The generated code will looks like: ``` /* 032 */ if (!initAgg0) { /* 033 */ initAgg0 = true; /* 034 */ /* 035 */ // initialize aggregation buffer /* 037 */ long bufValue2 = 0L; /* 038 */ /* 039 */ /* 040 */ // initialize Range /* 041 */ if (!range_initRange5) { /* 042 */ range_initRange5 = true; ... /* 071 */ } /* 072 */ /* 073 */ while (!range_overflow8 && range_number7 < range_partitionEnd6) { /* 074 */ long range_value9 = range_number7; /* 075 */ range_number7 += 1L; /* 076 */ if (range_number7 < range_value9 ^ 1L < 0) { /* 077 */ range_overflow8 = true; /* 078 */ } /* 079 */ /* 085 */ boolean primitive11 = false; /* 086 */ primitive11 = range_value9 > 1L; /* 087 */ if (!false && primitive11) { /* 092 */ // do aggregate and update aggregation buffer /* 099 */ long primitive17 = -1L; /* 100 */ primitive17 = bufValue2 + 1L; /* 101 */ bufValue2 = primitive17; /* 105 */ } /* 107 */ } /* 109 */ /* 110 */ // output the result /* 112 */ bufferHolder25.reset(); /* 114 */ rowWriter26.initialize(bufferHolder25, 1); /* 118 */ rowWriter26.write(0, bufValue2); /* 120 */ result24.pointTo(bufferHolder25.buffer, bufferHolder25.totalSize()); /* 121 */ currentRow = result24; /* 122 */ return; /* 124 */ } /* 125 */ ``` cc nongli Author: Davies Liu <davies@databricks.com> Closes #10840 from davies/gen_agg.
* [SPARK-12848][SQL] Change parsed decimal literal datatype from Double to DecimalHerman van Hovell2016-01-2028-59/+83
| | | | | | | | | | | | | | The current parser turns a decimal literal, for example ```12.1```, into a Double. The problem with this approach is that we convert an exact literal into a non-exact ```Double```. The PR changes this behavior, a Decimal literal is now converted into an extact ```BigDecimal```. The behavior for scientific decimals, for example ```12.1e01```, is unchanged. This will be converted into a Double. This PR replaces the ```BigDecimal``` literal by a ```Double``` literal, because the ```BigDecimal``` is the default now. You can use the double literal by appending a 'D' to the value, for instance: ```3.141527D``` cc davies rxin Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #10796 from hvanhovell/SPARK-12848.
* [SPARK-12888][SQL] benchmark the new hash expressionWenchen Fan2016-01-201-0/+104
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Benchmark it on 4 different schemas, the result: ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz Hash For simple: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- interpreted version 31.47 266.54 1.00 X codegen version 64.52 130.01 0.49 X ``` ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz Hash For normal: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- interpreted version 4068.11 0.26 1.00 X codegen version 1175.92 0.89 3.46 X ``` ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz Hash For array: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- interpreted version 9276.70 0.06 1.00 X codegen version 14762.23 0.04 0.63 X ``` ``` Intel(R) Core(TM) i7-4960HQ CPU 2.60GHz Hash For map: Avg Time(ms) Avg Rate(M/s) Relative Rate ------------------------------------------------------------------------------- interpreted version 58869.79 0.01 1.00 X codegen version 9285.36 0.06 6.34 X ``` Author: Wenchen Fan <wenchen@databricks.com> Closes #10816 from cloud-fan/hash-benchmark.
* [SPARK-12616][SQL] Making Logical Operator `Union` Support Arbitrary Number ↵gatorsmile2016-01-2020-122/+322
| | | | | | | | | | | | | | of Children The existing `Union` logical operator only supports two children. Thus, adding a new logical operator `Unions` which can have arbitrary number of children to replace the existing one. `Union` logical plan is a binary node. However, a typical use case for union is to union a very large number of input sources (DataFrames, RDDs, or files). It is not uncommon to union hundreds of thousands of files. In this case, our optimizer can become very slow due to the large number of logical unions. We should change the Union logical plan to support an arbitrary number of children, and add a single rule in the optimizer to collapse all adjacent `Unions` into a single `Unions`. Note that this problem doesn't exist in physical plan, because the physical `Unions` already supports arbitrary number of children. Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #10577 from gatorsmile/unionAllMultiChildren.
* [SPARK-7799][SPARK-12786][STREAMING] Add "streaming-akka" projectShixiong Zhu2016-01-2022-185/+601
| | | | | | | | | | | | | Include the following changes: 1. Add "streaming-akka" project and org.apache.spark.streaming.akka.AkkaUtils for creating an actorStream 2. Remove "StreamingContext.actorStream" and "JavaStreamingContext.actorStream" 3. Update the ActorWordCount example and add the JavaActorWordCount example 4. Make "streaming-zeromq" depend on "streaming-akka" and update the codes accordingly Author: Shixiong Zhu <shixiong@databricks.com> Closes #10744 from zsxwing/streaming-akka-2.
* [SPARK-12847][CORE][STREAMING] Remove StreamingListenerBus and post all ↵Shixiong Zhu2016-01-2014-231/+269
| | | | | | | | | | | | | | | Streaming events to the same thread as Spark events Including the following changes: 1. Add StreamingListenerForwardingBus to WrappedStreamingListenerEvent process events in `onOtherEvent` to StreamingListener 2. Remove StreamingListenerBus 3. Merge AsynchronousListenerBus and LiveListenerBus to the same class LiveListenerBus 4. Add `logEvent` method to SparkListenerEvent so that EventLoggingListener can use it to ignore WrappedStreamingListenerEvents Author: Shixiong Zhu <shixiong@databricks.com> Closes #10779 from zsxwing/streaming-listener.
* [SPARK-10263][ML] Add @Since annotation to ml.param and ml.*Takahashi Hiroshi2016-01-202-5/+42
| | | | | | | | | Add Since annotations to ml.param and ml.* Author: Takahashi Hiroshi <takahashi.hiroshi@lab.ntt.co.jp> Author: Hiroshi Takahashi <takahashi.hiroshi@lab.ntt.co.jp> Closes #8935 from taishi-oss/issue10263.
* [SPARK-12898] Consider having dummyCallSite for HiveTableScanRajesh Balamohan2016-01-201-3/+10
| | | | | | | | Currently, HiveTableScan runs with getCallSite which is really expensive and shows up when scanning through large table with partitions (e.g TPC-DS) which slows down the overall runtime of the job. It would be good to consider having dummyCallSite in HiveTableScan. Author: Rajesh Balamohan <rbalamohan@apache.org> Closes #10825 from rajeshbalamohan/SPARK-12898.
* [SPARK-12925][SQL] Improve HiveInspectors.unwrap for StringObjectIns…Rajesh Balamohan2016-01-201-1/+3
| | | | | | | | Text is in UTF-8 and converting it via "UTF8String.fromString" incurs decoding and encoding, which turns out to be expensive and redundant. Profiler snapshot details is attached in the JIRA (ref:https://issues.apache.org/jira/secure/attachment/12783331/SPARK-12925_profiler_cpu_samples.png) Author: Rajesh Balamohan <rbalamohan@apache.org> Closes #10848 from rajeshbalamohan/SPARK-12925.
* [SPARK-12230][ML] WeightedLeastSquares.fit() should handle division by zero ↵Imran Younus2016-01-202-7/+83
| | | | | | | | | | properly if standard deviation of target variable is zero. This fixes the behavior of WeightedLeastSquars.fit() when the standard deviation of the target variable is zero. If the fitIntercept is true, there is no need to train. Author: Imran Younus <iyounus@us.ibm.com> Closes #10274 from iyounus/SPARK-12230_bug_fix_in_weighted_least_squares.
* [SPARK-11295][PYSPARK] Add packages to JUnit output for Python testsGábor Lipták2016-01-205-11/+19
| | | | | | | | | This is #9263 from gliptak (improving grouping/display of test case results) with a small fix of bisecting k-means unit test. Author: Gábor Lipták <gliptak@gmail.com> Author: Xiangrui Meng <meng@databricks.com> Closes #10850 from mengxr/SPARK-11295.
* [SPARK-6519][ML] Add spark.ml API for bisecting k-meansYu ISHIKAWA2016-01-202-0/+281
| | | | | | Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #9604 from yu-iskw/SPARK-6519.