aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-9913] [MLLIB] LDAUtils should be privateXiangrui Meng2015-08-121-1/+1
| | | | | | | | feynmanliang Author: Xiangrui Meng <meng@databricks.com> Closes #8142 from mengxr/SPARK-9913.
* [SPARK-9894] [SQL] Json writer should handle MapData.Yin Huai2015-08-123-35/+83
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-9894 Author: Yin Huai <yhuai@databricks.com> Closes #8137 from yhuai/jsonMapData.
* [SPARK-9826] [CORE] Fix cannot use custom classes in log4j.propertiesMichel Lemay2015-08-1216-252/+307
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Refactor Utils class and create ShutdownHookManager. NOTE: Wasn't able to run /dev/run-tests on windows machine. Manual tests were conducted locally using custom log4j.properties file with Redis appender and logstash formatter (bundled in the fat-jar submitted to spark) ex: log4j.rootCategory=WARN,console,redis log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.apache.spark.graphx.Pregel=INFO log4j.appender.redis=com.ryantenney.log4j.FailoverRedisAppender log4j.appender.redis.endpoints=hostname:port log4j.appender.redis.key=mykey log4j.appender.redis.alwaysBatch=false log4j.appender.redis.layout=net.logstash.log4j.JSONEventLayoutV1 Author: michellemay <mlemay@gmail.com> Closes #8109 from michellemay/SPARK-9826.
* [SPARK-9092] Fixed incompatibility when both num-executors and dynamic...Niranjan Padmanabhan2015-08-1214-26/+64
| | | | | | | | … allocation are set. Now, dynamic allocation is set to false when num-executors is explicitly specified as an argument. Consequently, executorAllocationManager in not initialized in the SparkContext. Author: Niranjan Padmanabhan <niranjan.padmanabhan@cloudera.com> Closes #7657 from neurons/SPARK-9092.
* [SPARK-9907] [SQL] Python crc32 is mistakenly calling md5Reynold Xin2015-08-121-2/+2
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #8138 from rxin/SPARK-9907.
* [SPARK-8967] [DOC] add Since annotationXiangrui Meng2015-08-121-0/+28
| | | | | | | | | | | | | | | | | | | | Add `Since` as a Scala annotation. The benefit is that we can use it without having explicit JavaDoc. This is useful for inherited methods. The limitation is that is doesn't show up in the generated Java API documentation. This might be fixed by modifying genjavadoc. I think we could leave it as a TODO. This is how the generated Scala doc looks: `since` JavaDoc tag: ![screen shot 2015-08-11 at 10 00 37 pm](https://cloud.githubusercontent.com/assets/829644/9230761/fa72865c-40d8-11e5-807e-0f3c815c5acd.png) `Since` annotation: ![screen shot 2015-08-11 at 10 00 28 pm](https://cloud.githubusercontent.com/assets/829644/9230764/0041d7f4-40d9-11e5-8124-c3f3e5d5b31f.png) rxin Author: Xiangrui Meng <meng@databricks.com> Closes #8131 from mengxr/SPARK-8967.
* [SPARK-9789] [ML] Added logreg threshold param backJoseph K. Bradley2015-08-126-76/+199
| | | | | | | | | | Reinstated LogisticRegression.threshold Param for binary compatibility. Param thresholds overrides threshold, if set. CC: mengxr dbtsai feynmanliang Author: Joseph K. Bradley <joseph@databricks.com> Closes #8079 from jkbradley/logreg-reinstate-threshold.
* [SPARK-9766] [ML] [PySpark] check and add miss docs for PySpark MLYanbo Liang2015-08-124-8/+20
| | | | | | | | Check and add miss docs for PySpark ML (this issue only check miss docs for o.a.s.ml not o.a.s.mllib). Author: Yanbo Liang <ybliang8@gmail.com> Closes #8059 from yanboliang/SPARK-9766.
* [SPARK-9726] [PYTHON] PySpark DF join no longer accepts on=NoneBrennan Ashton2015-08-121-2/+1
| | | | | | | | | | | rxin First pull request for Spark so let me know if I am missing anything The contribution is my original work and I license the work to the project under the project's open source license. Author: Brennan Ashton <bashton@brennanashton.com> Closes #8016 from btashton/patch-1.
* [SPARK-9847] [ML] Modified copyValues to distinguish between default, ↵Joseph K. Bradley2015-08-122-3/+24
| | | | | | | | | | | | | explicit param values From JIRA: Currently, Params.copyValues copies default parameter values to the paramMap of the target instance, rather than the defaultParamMap. It should copy to the defaultParamMap because explicitly setting a parameter can change the semantics. This issue arose in SPARK-9789, where 2 params "threshold" and "thresholds" for LogisticRegression can have mutually exclusive values. If thresholds is set, then fit() will copy the default value of threshold as well, easily resulting in inconsistent settings for the 2 params. CC: mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #8115 from jkbradley/copyvalues-fix.
* [SPARK-9804] [HIVE] Use correct value for isSrcLocal parameter.Marcelo Vanzin2015-08-121-3/+10
| | | | | | | | | | If the correct parameter is not provided, Hive will run into an error because it calls methods that are specific to the local filesystem to copy the data. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #8086 from vanzin/SPARK-9804.
* [SPARK-9747] [SQL] Avoid starving an unsafe operator in aggregationAndrew Or2015-08-127-76/+201
| | | | | | | | | | | | This is the sister patch to #8011, but for aggregation. In a nutshell: create the `TungstenAggregationIterator` before computing the parent partition. Internally this creates a `BytesToBytesMap` which acquires a page in the constructor as of this patch. This ensures that the aggregation operator is not starved since we reserve at least 1 page in advance. rxin yhuai Author: Andrew Or <andrew@databricks.com> Closes #8038 from andrewor14/unsafe-starve-memory-agg.
* [SPARK-7583] [MLLIB] User guide update for RegexTokenizerYuhao Yang2015-08-121-11/+30
| | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-7583 User guide update for RegexTokenizer Author: Yuhao Yang <hhbyyh@gmail.com> Closes #7828 from hhbyyh/regexTokenizerDoc.
* [SPARK-9795] Dynamic allocation: avoid double counting when killing same ↵Andrew Or2015-08-122-4/+27
| | | | | | | | | | | | | | | | | | | executor twice This is based on KaiXinXiaoLei's changes in #7716. The issue is that when someone calls `sc.killExecutor("1")` on the same executor twice quickly, then the executor target will be adjusted downwards by 2 instead of 1 even though we're only actually killing one executor. In certain cases where we don't adjust the target back upwards quickly, we'll end up with jobs hanging. This is a common danger because there are many places where this is called: - `HeartbeatReceiver` kills an executor that has not been sending heartbeats - `ExecutorAllocationManager` kills an executor that has been idle - The user code might call this, which may interfere with the previous callers While it's not clear whether this fixes SPARK-9745, fixing this potential race condition seems like a strict improvement. I've added a regression test to illustrate the issue. Author: Andrew Or <andrew@databricks.com> Closes #8078 from andrewor14/da-double-kill.
* [SPARK-8625] [CORE] Propagate user exceptions in tasks back to driverTom White2015-08-1212-34/+165
| | | | | | | | | | | | | | This allows clients to retrieve the original exception from the cause field of the SparkException that is thrown by the driver. If the original exception is not in fact Serializable then it will not be returned, but the message and stacktrace will be. (All Java Throwables implement the Serializable interface, but this is no guarantee that a particular implementation can actually be serialized.) Author: Tom White <tom@cloudera.com> Closes #7014 from tomwhite/propagate-user-exceptions.
* [SPARK-9407] [SQL] Relaxes Parquet ValidTypeMap to allow ENUM predicates to ↵Cheng Lian2015-08-1216-136/+374
| | | | | | | | | | | | | | | | be pushed down This PR adds a hacky workaround for PARQUET-201, and should be removed once we upgrade to parquet-mr 1.8.1 or higher versions. In Parquet, not all types of columns can be used for filter push-down optimization. The set of valid column types is controlled by `ValidTypeMap`. Unfortunately, in parquet-mr 1.7.0 and prior versions, this limitation is too strict, and doesn't allow `BINARY (ENUM)` columns to be pushed down. On the other hand, `BINARY (ENUM)` is commonly seen in Parquet files written by libraries like `parquet-avro`. This restriction is problematic for Spark SQL, because Spark SQL doesn't have a type that maps to Parquet `BINARY (ENUM)` directly, and always converts `BINARY (ENUM)` to Catalyst `StringType`. Thus, a predicate involving a `BINARY (ENUM)` is recognized as one involving a string field instead and can be pushed down by the query optimizer. Such predicates are actually perfectly legal except that it fails the `ValidTypeMap` check. The workaround added here is relaxing `ValidTypeMap` to include `BINARY (ENUM)`. I also took the chance to simplify `ParquetCompatibilityTest` a little bit when adding regression test. Author: Cheng Lian <lian@databricks.com> Closes #8107 from liancheng/spark-9407/parquet-enum-filter-push-down.
* [SPARK-9182] [SQL] Filters are not passed through to jdbc sourceYijie Shen2015-08-123-3/+63
| | | | | | | | | | This PR fixes unable to push filter down to JDBC source caused by `Cast` during pattern matching. While we are comparing columns of different type, there's a big chance we need a cast on the column, therefore not match the pattern directly on Attribute and would fail to push down. Author: Yijie Shen <henry.yijieshen@gmail.com> Closes #8049 from yjshen/jdbc_pushdown.
* [SPARK-9575] [MESOS] Add docuemntation around Mesos shuffle service.Timothy Chen2015-08-111-0/+14
| | | | | | | | andrewor14 Author: Timothy Chen <tnachen@gmail.com> Closes #7907 from tnachen/mesos_shuffle.
* [SPARK-8798] [MESOS] Allow additional uris to be fetched with mesosTimothy Chen2015-08-115-0/+27
| | | | | | | | Some users like to download additional files in their sandbox that they can refer to from their spark program, or even later mount these files to another directory. Author: Timothy Chen <tnachen@gmail.com> Closes #7195 from tnachen/mesos_files.
* [SPARK-9426] [WEBUI] Job page DAG visualization is not shownCarson Wang2015-08-112-5/+6
| | | | | | | | | | To reproduce the issue, go to the stage page and click DAG Visualization once, then go to the job page to show the job DAG visualization. You will only see the first stage of the job. Root cause: the java script use local storage to remember your selection. Once you click the stage DAG visualization, the local storage set `expand-dag-viz-arrow-stage` to true. When you go to the job page, the js checks `expand-dag-viz-arrow-stage` in the local storage first and will try to show stage DAG visualization on the job page. To fix this, I set an id to the DAG span to differ job page and stage page. In the js code, we check the id and local storage together to make sure we show the correct DAG visualization. Author: Carson Wang <carson.wang@intel.com> Closes #8104 from carsonwang/SPARK-9426.
* [SPARK-9829] [WEBUI] Display the update value for peak execution memoryzsxwing2015-08-111-1/+1
| | | | | | | | | | The peak execution memory is not correct because it shows the sum of finished tasks' values when a task finishes. This PR fixes it by using the update value rather than the accumulator value. Author: zsxwing <zsxwing@gmail.com> Closes #8121 from zsxwing/SPARK-9829.
* [SPARK-9806] [WEB UI] Don't share ReplayListenerBus between multiple ↵Rohit Agarwal2015-08-111-1/+1
| | | | | | | | applications Author: Rohit Agarwal <rohita@qubole.com> Closes #8088 from mindprince/SPARK-9806.
* [SPARK-8366] maxNumExecutorsNeeded should properly handle failed tasksxutingjun2015-08-112-10/+34
| | | | | | | Author: xutingjun <xutingjun@huawei.com> Author: meiyoula <1039320815@qq.com> Closes #6817 from XuTingjun/SPARK-8366.
* [SPARK-9854] [SQL] RuleExecutor.timeMap should be thread-safeJosh Rosen2015-08-111-6/+9
| | | | | | | | | | `RuleExecutor.timeMap` is currently a non-thread-safe mutable HashMap; this can lead to infinite loops if multiple threads are concurrently modifying the map. I believe that this is responsible for some hangs that I've observed in HiveQuerySuite. This patch addresses this by using a Guava `AtomicLongMap`. Author: Josh Rosen <joshrosen@databricks.com> Closes #8120 from JoshRosen/rule-executor-time-map-fix.
* [SPARK-9831] [SQL] fix serialization with empty broadcastDavies Liu2015-08-112-1/+18
| | | | | | | | Author: Davies Liu <davies@databricks.com> Closes #8117 from davies/fix_serialization and squashes the following commits: d21ac71 [Davies Liu] fix serialization with empty broadcast
* [SPARK-9713] [ML] Document SparkR MLlib glm() integration in Spark 1.5Eric Liang2015-08-113-7/+42
| | | | | | | | | | This documents the use of R model formulae in the SparkR guide. Also fixes some bugs in the R api doc. mengxr Author: Eric Liang <ekl@databricks.com> Closes #8085 from ericl/docs.
* [SPARK-1517] Refactor release scripts to facilitate nightly publishingPatrick Wendell2015-08-113-267/+400
| | | | | | | | | | | | | | | | | | This update contains some code changes to the release scripts that allow easier nightly publishing. I've been using these new scripts on Jenkins for cutting and publishing nightly snapshots for the last month or so, and it has been going well. I'd like to get them merged back upstream so this can be maintained by the community. The main changes are: 1. Separates the release tagging from various build possibilities for an already tagged release (`release-tag.sh` and `release-build.sh`). 2. Allow for injecting credentials through the environment, including GPG keys. This is then paired with secure key injection in Jenkins. 3. Support for copying build results to a remote directory, and also "rotating" results, e.g. the ability to keep the last N copies of binary or doc builds. I'm happy if anyone wants to take a look at this - it's not user facing but an internal utility used for generating releases. Author: Patrick Wendell <patrick@databricks.com> Closes #7411 from pwendell/release-script-updates and squashes the following commits: 74f9beb [Patrick Wendell] Moving maven build command to a variable 233ce85 [Patrick Wendell] [SPARK-1517] Refactor release scripts to facilitate nightly publishing
* [SPARK-9649] Fix flaky test MasterSuite again - disable RESTAndrew Or2015-08-112-0/+2
| | | | | | | | The REST server is not actually used in most tests and so we can disable it. It is a source of flakiness because it tries to bind to a specific port in vain. There was also some code that avoided the shuffle service in tests. This is actually not necessary because the shuffle service is already off by default. Author: Andrew Or <andrew@databricks.com> Closes #8084 from andrewor14/fix-master-suite-again.
* [SPARK-9849] [SQL] DirectParquetOutputCommitter qualified name should be ↵Reynold Xin2015-08-112-1/+33
| | | | | | | | | | backward compatible DirectParquetOutputCommitter was moved in SPARK-9763. However, users can explicitly set the class as a config option, so we must be able to resolve the old committer qualified name. Author: Reynold Xin <rxin@databricks.com> Closes #8114 from rxin/SPARK-9849.
* [SPARK-9074] [LAUNCHER] Allow arbitrary Spark args to be set.Marcelo Vanzin2015-08-113-3/+150
| | | | | | | | | | | | | | This change allows any Spark argument to be added to the app to be started using SparkLauncher. Known arguments are properly validated, while unknown arguments are allowed so that the library can launch newer Spark versions (in case SPARK_HOME points at one). Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #7975 from vanzin/SPARK-9074 and squashes the following commits: b5e451a [Marcelo Vanzin] [SPARK-9074] [launcher] Allow arbitrary Spark args to be set.
* [HOTFIX] Fix style error caused by 017b5deAndrew Or2015-08-111-1/+1
|
* [SPARK-8925] [MLLIB] Add @since tags to mllib.utilSudhakar Thota2015-08-111-1/+21
| | | | | | | | | Went thru the history of changes the file MLUtils.scala and picked up the version that the change went in. Author: Sudhakar Thota <sudhakarthota@yahoo.com> Author: Sudhakar Thota <sudhakarthota@sudhakars-mbp-2.usca.ibm.com> Closes #7436 from sthota2014/SPARK-8925_thotas.
* [SPARK-9788] [MLLIB] Fix LDA Binary CompatibilityFeynman Liang2015-08-114-24/+46
| | | | | | | | | | | | | | | | 1. Add “asymmetricDocConcentration” and revert docConcentration changes. If the (internal) doc concentration vector is a single value, “getDocConcentration" returns it. If it is a constant vector, getDocConcentration returns the first item, and fails otherwise. 2. Give `LDAModel.gammaShape` a default value in `LDAModel` concrete class constructors. jkbradley Author: Feynman Liang <fliang@databricks.com> Closes #8077 from feynmanliang/SPARK-9788 and squashes the following commits: 6b07bc8 [Feynman Liang] Code review changes 9d6a71e [Feynman Liang] Add asymmetricAlpha alias bf4e685 [Feynman Liang] Asymmetric docConcentration 4cab972 [Feynman Liang] Default gammaShape
* Closes #1290Xiangrui Meng2015-08-110-0/+0
| | | | Closes #4934
* [SPARK-9824] [CORE] Fix the issue that InternalAccumulator leaks WeakReferencezsxwing2015-08-113-11/+16
| | | | | | | | | | `InternalAccumulator.create` doesn't call `registerAccumulatorForCleanup` to register itself with ContextCleaner, so `WeakReference`s for these accumulators in `Accumulators.originals` won't be removed. This PR added `registerAccumulatorForCleanup` for internal accumulators to avoid the memory leak. Author: zsxwing <zsxwing@gmail.com> Closes #8108 from zsxwing/internal-accumulators-leak.
* [SPARK-9814] [SQL] EqualNotNull not passing to data sourceshyukjinkwon2015-08-113-0/+15
| | | | | | | Author: hyukjinkwon <gurwls223@gmail.com> Author: 권혁진 <gurwls223@gmail.com> Closes #8096 from HyukjinKwon/master.
* [SPARK-7726] Add import so Scaladoc doesn't fail.Patrick Wendell2015-08-111-0/+3
| | | | | | | | | | This is another import needed so Scala 2.11 doc generation doesn't fail. See SPARK-7726 for more detail. I tested this locally and the 2.11 install goes from failing to succeeding with this patch. Author: Patrick Wendell <patrick@databricks.com> Closes #8095 from pwendell/scaladoc.
* [SPARK-9750] [MLLIB] Improve equals on SparseMatrix and DenseMatrixFeynman Liang2015-08-112-2/+24
| | | | | | | | | | | | | | | | | Adds unit test for `equals` on `mllib.linalg.Matrix` class and `equals` to both `SparseMatrix` and `DenseMatrix`. Supports equality testing between `SparseMatrix` and `DenseMatrix`. mengxr Author: Feynman Liang <fliang@databricks.com> Closes #8042 from feynmanliang/SPARK-9750 and squashes the following commits: bb70d5e [Feynman Liang] Breeze compare for dense matrices as well, in case other is sparse ab6f3c8 [Feynman Liang] Sparse matrix compare for equals 22782df [Feynman Liang] Add equality based on matrix semantics, not representation 78f9426 [Feynman Liang] Add casts 43d28fa [Feynman Liang] Fix failing test 6416fa0 [Feynman Liang] Add failing sparse matrix equals tests
* [SPARK-9646] [SQL] Add metrics for all join and aggregate operatorszsxwing2015-08-1127-107/+847
| | | | | | | | | | | | | | | | | | | This PR added metrics for all join and aggregate operators. However, I found the metrics may be confusing in the following two case: 1. The iterator is not totally consumed and the metric values will be less. 2. Recreating the iterators will make metric values look bigger than the size of the input source, such as `CartesianProduct`. Author: zsxwing <zsxwing@gmail.com> Closes #8060 from zsxwing/sql-metrics and squashes the following commits: 40f3fc1 [zsxwing] Mark LongSQLMetric private[metric] to avoid using incorrectly and leak memory b1b9071 [zsxwing] Merge branch 'master' into sql-metrics 4bef25a [zsxwing] Add metrics for SortMergeOuterJoin 95ccfc6 [zsxwing] Merge branch 'master' into sql-metrics 67cb4dd [zsxwing] Add metrics for Project and TungstenProject; remove metrics from PhysicalRDD and LocalTableScan 0eb47d4 [zsxwing] Merge branch 'master' into sql-metrics dd9d932 [zsxwing] Avoid creating new Iterators 589ea26 [zsxwing] Add metrics for all join and aggregate operators
* [SPARK-9572] [STREAMING] [PYSPARK] Added ↵Tathagata Das2015-08-113-15/+177
| | | | | | | | | | | | | | | | | | | | | | | StreamingContext.getActiveOrCreate() in Python Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8080 from tdas/SPARK-9572 and squashes the following commits: 64a231d [Tathagata Das] Fix based on comments 741a0d0 [Tathagata Das] Fixed style f4f094c [Tathagata Das] Tweaked test 9afcdbe [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9572 e21488d [Tathagata Das] Minor update 1a371d9 [Tathagata Das] Addressed comments. 60479da [Tathagata Das] Fixed indent 9c2da9c [Tathagata Das] Fixed bugs b5bd32c [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9572 b55b348 [Tathagata Das] Removed prints 5781728 [Tathagata Das] Fix style issues b711214 [Tathagata Das] Reverted run-tests.py 643b59d [Tathagata Das] Revert unnecessary change 150e58c [Tathagata Das] Added StreamingContext.getActiveOrCreate() in Python
* [SPARK-8764] [ML] string indexer should take option to handle unseen valuesHolden Karau2015-08-114-4/+73
| | | | | | | | | | | | | | | | | | | | | | As a precursor to adding a public constructor add an option to handle unseen values by skipping rather than throwing an exception (default remains throwing an exception), Author: Holden Karau <holden@pigscanfly.ca> Closes #7266 from holdenk/SPARK-8764-string-indexer-should-take-option-to-handle-unseen-values and squashes the following commits: 38a4de9 [Holden Karau] fix long line 045bf22 [Holden Karau] Add a second b entry so b gets 0 for sure 81dd312 [Holden Karau] Update the docs for handleInvalid param to be more descriptive 7f37f6e [Holden Karau] remove extra space (scala style) 414e249 [Holden Karau] And switch to using handleInvalid instead of skipInvalid 1e53f9b [Holden Karau] update the param (codegen side) 7a22215 [Holden Karau] fix typo 100a39b [Holden Karau] Merge in master aa5b093 [Holden Karau] Since we filter we should never go down this code path if getSkipInvalid is true 75ffa69 [Holden Karau] Remove extra newline d69ef5e [Holden Karau] Add a test b5734be [Holden Karau] Add support for unseen labels afecd4e [Holden Karau] Add a param to skip invalid entries.
* [SPARK-8345] [ML] Add an SQL node as a feature transformerYanbo Liang2015-08-112-0/+116
| | | | | | | | | | | | | | Implements the transforms which are defined by SQL statement. Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' where '__THIS__' represents the underlying table of the input dataset. Author: Yanbo Liang <ybliang8@gmail.com> Closes #7465 from yanboliang/spark-8345 and squashes the following commits: b403fcb [Yanbo Liang] address comments 0d4bb15 [Yanbo Liang] a better transformSchema() implementation 51eb9e7 [Yanbo Liang] Add an SQL node as a feature transformer
* Fix comment errorJeff Zhang2015-08-111-1/+1
| | | | | | | | API is updated but its doc comment is not updated. Author: Jeff Zhang <zjffdu@apache.org> Closes #8097 from zjffdu/dev.
* [SPARK-9785] [SQL] HashPartitioning compatibility should consider expression ↵Josh Rosen2015-08-112-10/+60
| | | | | | | | | | | | | | | | ordering HashPartitioning compatibility is currently defined w.r.t the _set_ of expressions, but the ordering of those expressions matters when computing hash codes; this could lead to incorrect answers if we mistakenly avoided a shuffle based on the assumption that HashPartitionings with the same expressions in different orders will produce equivalent row hashcodes. The first commit adds a regression test which illustrates this problem. The fix for this is simple: make `HashPartitioning.compatibleWith` and `HashPartitioning.guarantees` sensitive to the expression ordering (i.e. do not perform set comparison). Author: Josh Rosen <joshrosen@databricks.com> Closes #8074 from JoshRosen/hashpartitioning-compatiblewith-fixes and squashes the following commits: b61412f [Josh Rosen] Demonstrate that I haven't cheated in my fix 0b4d7d9 [Josh Rosen] Update so that clusteringSet is only used in satisfies(). dc9c9d7 [Josh Rosen] Add failing regression test for SPARK-9785
* [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.Reynold Xin2015-08-1139-499/+371
| | | | | | | | | | PlatformDependent.UNSAFE is way too verbose. Author: Reynold Xin <rxin@databricks.com> Closes #8094 from rxin/SPARK-9815 and squashes the following commits: 229b603 [Reynold Xin] [SPARK-9815] Rename PlatformDependent.UNSAFE -> Platform.
* [SPARK-9727] [STREAMING] [BUILD] Updated streaming kinesis SBT project name ↵Tathagata Das2015-08-113-5/+5
| | | | | | | | | | to be more consistent Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #8092 from tdas/SPARK-9727 and squashes the following commits: b1b01fd [Tathagata Das] Updated streaming kinesis project name
* [SPARK-9810] [BUILD] Remove individual commit messages from the squash ↵Reynold Xin2015-08-111-5/+1
| | | | | | | | | | | | | | commit message For more information, please see the JIRA ticket and the associated dev list discussion. https://issues.apache.org/jira/browse/SPARK-9810 http://apache-spark-developers-list.1001551.n3.nabble.com/discuss-Removing-individual-commit-messages-from-the-squash-commit-message-td13295.html Author: Reynold Xin <rxin@databricks.com> Closes #8091 from rxin/SPARK-9810.
* [SPARK-9640] [STREAMING] [TEST] Do not run Python Kinesis tests when the ↵Tathagata Das2015-08-101-12/+44
| | | | | | | | | | | | | | | | | | | | Kinesis assembly JAR has not been generated Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7961 from tdas/SPARK-9640 and squashes the following commits: 974ce19 [Tathagata Das] Undo changes related to SPARK-9727 004ae26 [Tathagata Das] style fixes 9bbb97d [Tathagata Das] Minor style fies e6a677e [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-9640 ca90719 [Tathagata Das] Removed extra line ba9cfc7 [Tathagata Das] Improved kinesis test selection logic 88d59bd [Tathagata Das] updated test modules 871fcc8 [Tathagata Das] Fixed SparkBuild 94be631 [Tathagata Das] Fixed style b858196 [Tathagata Das] Fixed conditions and few other things based on PR comments. e292e64 [Tathagata Das] Added filters for Kinesis python tests
* [SPARK-9729] [SPARK-9363] [SQL] Use sort merge join for left and right outer ↵Josh Rosen2015-08-1013-319/+1165
| | | | | | | | | | | | | | | | | | | | | | | | | join This patch adds a new `SortMergeOuterJoin` operator that performs left and right outer joins using sort merge join. It also refactors `SortMergeJoin` in order to improve performance and code clarity. Along the way, I also performed a couple pieces of minor cleanup and optimization: - Rename the `HashJoin` physical planner rule to `EquiJoinSelection`, since it's also used for non-hash joins. - Rewrite the comment at the top of `HashJoin` to better explain the precedence for choosing join operators. - Update `JoinSuite` to use `SqlTestUtils.withConf` for changing SQLConf settings. This patch incorporates several ideas from adrian-wang's patch, #5717. Closes #5717. <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/7904) <!-- Reviewable:end --> Author: Josh Rosen <joshrosen@databricks.com> Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #7904 from JoshRosen/outer-join-smj and squashes 1 commits.
* [SPARK-9340] [SQL] Fixes converting unannotated Parquet listsDamian Guy2015-08-1114-33/+247
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR is inspired by #8063 authored by dguy. Especially, testing Parquet files added here are all taken from that PR. **Committer who merges this PR should attribute it to "Damian Guy <damian.guygmail.com>".** ---- SPARK-6776 and SPARK-6777 followed `parquet-avro` to implement backwards-compatibility rules defined in `parquet-format` spec. However, both Spark SQL and `parquet-avro` neglected the following statement in `parquet-format`: > This does not affect repeated fields that are not annotated: A repeated field that is neither contained by a `LIST`- or `MAP`-annotated group nor annotated by `LIST` or `MAP` should be interpreted as a required list of required elements where the element type is the type of the field. One of the consequences is that, Parquet files generated by `parquet-protobuf` containing unannotated repeated fields are not correctly converted to Catalyst arrays. This PR fixes this issue by 1. Handling unannotated repeated fields in `CatalystSchemaConverter`. 2. Converting this kind of special repeated fields to Catalyst arrays in `CatalystRowConverter`. Two special converters, `RepeatedPrimitiveConverter` and `RepeatedGroupConverter`, are added. They delegate actual conversion work to a child `elementConverter` and accumulates elements in an `ArrayBuffer`. Two extra methods, `start()` and `end()`, are added to `ParentContainerUpdater`. So that they can be used to initialize new `ArrayBuffer`s for unannotated repeated fields, and propagate converted array values to upstream. Author: Cheng Lian <lian@databricks.com> Closes #8070 from liancheng/spark-9340/unannotated-parquet-list and squashes the following commits: ace6df7 [Cheng Lian] Moves ParquetProtobufCompatibilitySuite f1c7bfd [Cheng Lian] Updates .rat-excludes 420ad2b [Cheng Lian] Fixes converting unannotated Parquet lists