aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-4134][CORE] Lower severity of some executor loss logs.Marcelo Vanzin2015-11-195-24/+45
| | | | | | | | | Don't log ERROR messages when executors are explicitly killed or when the exit reason is not yet known. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9780 from vanzin/SPARK-11789.
* [SPARK-11275][SQL] Incorrect results when using rollup/cubeAndrew Ray2015-11-193-34/+90
| | | | | | | | | | | | | | | | | Fixes bug with grouping sets (including cube/rollup) where aggregates that included grouping expressions would return the wrong (null) result. Also simplifies the analyzer rule a bit and leaves column pruning to the optimizer. Added multiple unit tests to DataFrameAggregateSuite and verified it passes hive compatibility suite: ``` build/sbt -Phive -Dspark.hive.whitelist='groupby.*_grouping.*' 'test-only org.apache.spark.sql.hive.execution.HiveCompatibilitySuite' ``` This is an alternative to pr https://github.com/apache/spark/pull/9419 but I think its better as it simplifies the analyzer rule instead of adding another special case to it. Author: Andrew Ray <ray.andrew@gmail.com> Closes #9815 from aray/groupingset-agg-fix.
* [SPARK-11746][CORE] Use cache-aware method dependencieshushan2015-11-191-1/+1
| | | | | | | | a small change Author: hushan <hushan@xiaomi.com> Closes #9691 from suyanNone/unify-getDependency.
* [SPARK-11828][CORE] Register DAGScheduler metrics source after app id is known.Marcelo Vanzin2015-11-192-3/+2
| | | | | | Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9820 from vanzin/SPARK-11828.
* [SPARK-11799][CORE] Make it explicit in executor logs that uncaught e…Srinivasa Reddy Vundela2015-11-191-1/+5
| | | | | | | | | | …xceptions are thrown during executor shutdown This commit will make sure that when uncaught exceptions are prepended with [Container in shutdown] when JVM is shutting down. Author: Srinivasa Reddy Vundela <vsr@cloudera.com> Closes #9809 from vundela/master_11799.
* [SPARK-11831][CORE][TESTS] Use port 0 to avoid port conflicts in testsShixiong Zhu2015-11-192-14/+14
| | | | | | | | Use port 0 to fix port-contention-related flakiness Author: Shixiong Zhu <shixiong@databricks.com> Closes #9841 from zsxwing/SPARK-11831.
* [SPARK-11858][SQL] Move sql.columnar into sql.execution.Reynold Xin2015-11-1930-147/+155
| | | | | | | | In addition, tightened visibility of a lot of classes in the columnar package from private[sql] to private[columnar]. Author: Reynold Xin <rxin@databricks.com> Closes #9842 from rxin/SPARK-11858.
* [SPARK-11812][PYSPARK] invFunc=None works properly with python's ↵David Tolpin2015-11-192-3/+14
| | | | | | | | | | | | | reduceByKeyAndWindow invFunc is optional and can be None. Instead of invFunc (the parameter) invReduceFunc (a local function) was checked for trueness (that is, not None, in this context). A local function is never None, thus the case of invFunc=None (a common one when inverse reduction is not defined) was treated incorrectly, resulting in loss of data. In addition, the docstring used wrong parameter names, also fixed. Author: David Tolpin <david.tolpin@gmail.com> Closes #9775 from dtolpin/master.
* [SPARK-11778][SQL] parse table name before it is passed to lookupRelationHuaxin Gao2015-11-192-1/+12
| | | | | | | | | Fix a bug in DataFrameReader.table (table with schema name such as "db_name.table" doesn't work) Use SqlParser.parseTableIdentifier to parse the table name before lookupRelation. Author: Huaxin Gao <huaxing@oc0558782468.ibm.com> Closes #9773 from huaxingao/spark-11778.
* [SPARK-11750][SQL] revert SPARK-11727 and code clean upWenchen Fan2015-11-1914-1118/+364
| | | | | | | | After some experiment, I found it's not convenient to have separate encoder builders: `FlatEncoder` and `ProductEncoder`. For example, when create encoders for `ScalaUDF`, we have no idea if the type `T` is flat or not. So I revert the splitting change in https://github.com/apache/spark/pull/9693, while still keeping the bug fixes and tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #9726 from cloud-fan/follow.
* [SPARK-11848][SQL] Support EXPLAIN in DataSet APIsgatorsmile2015-11-192-22/+22
| | | | | | | | | | When debugging DataSet API, I always need to print the logical and physical plans. I am wondering if we should provide a simple API for EXPLAIN? Author: gatorsmile <gatorsmile@gmail.com> Closes #9832 from gatorsmile/explainDS.
* [SPARK-11633][SQL] LogicalRDD throws TreeNode Exception : Failed to Copy Nodegatorsmile2015-11-192-0/+18
| | | | | | | | | | | | | When handling self joins, the implementation did not consider the case insensitivity of HiveContext. It could cause an exception as shown in the JIRA: ``` TreeNodeException: Failed to copy node. ``` The fix is low risk. It avoids unnecessary attribute replacement. It should not affect the existing behavior of self joins. Also added the test case to cover this case. Author: gatorsmile <gatorsmile@gmail.com> Closes #9762 from gatorsmile/joinMakeCopy.
* [SPARK-11830][CORE] Make NettyRpcEnv bind to the specified hostzsxwing2015-11-194-10/+28
| | | | | | | | | | | | | This PR includes the following change: 1. Bind NettyRpcEnv to the specified host 2. Fix the port information in the log for NettyRpcEnv. 3. Fix the service name of NettyRpcEnv. Author: zsxwing <zsxwing@gmail.com> Author: Shixiong Zhu <shixiong@databricks.com> Closes #9821 from zsxwing/SPARK-11830.
* [SPARK-11840][SQL] Restore the 1.5's behavior of planning a single distinct ↵Yin Huai2015-11-191-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | aggregation. The impact of this change is for a query that has a single distinct column and does not have any grouping expression like `SELECT COUNT(DISTINCT a) FROM table` The plan will be changed from ``` AGG-2 (count distinct) Shuffle to a single reducer Partial-AGG-2 (count distinct) AGG-1 (grouping on a) Shuffle by a Partial-AGG-1 (grouping on 1) ``` to the following one (1.5 uses this) ``` AGG-2 AGG-1 (grouping on a) Shuffle to a single reducer Partial-AGG-1(grouping on a) ``` The first plan is more robust. However, to better benchmark the impact of this change, we should use 1.5's plan and use the conf of `spark.sql.specializeSingleDistinctAggPlanning` to control the plan. Author: Yin Huai <yhuai@databricks.com> Closes #9828 from yhuai/distinctRewriter.
* [SPARK-11849][SQL] Analyzer should replace current_date and ↵Reynold Xin2015-11-192-5/+60
| | | | | | | | | | | | current_timestamp with literals We currently rely on the optimizer's constant folding to replace current_timestamp and current_date. However, this can still result in different values for different instances of current_timestamp/current_date if the optimizer is not running fast enough. A better solution is to replace these functions in the analyzer in one shot. Author: Reynold Xin <rxin@databricks.com> Closes #9833 from rxin/SPARK-11849.
* [SPARK-11339][SPARKR] Document the list of functions in R base package that ↵felixcheung2015-11-186-6/+77
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are masked by functions with same name in SparkR Added tests for function that are reported as masked, to make sure the base:: or stats:: function can be called. For those we can't call, added them to SparkR programming guide. It would seem to me `table, sample, subset, filter, cov` not working are not actually expected - I investigated/experimented with them but couldn't get them to work. It looks like as they are defined in base or stats they are missing the S3 generic, eg. ``` > methods("transform") [1] transform,ANY-method transform.data.frame [3] transform,DataFrame-method transform.default see '?methods' for accessing help and source code > methods("subset") [1] subset.data.frame subset,DataFrame-method subset.default [4] subset.matrix see '?methods' for accessing help and source code Warning message: In .S3methods(generic.function, class, parent.frame()) : function 'subset' appears not to be S3 generic; found functions that look like S3 methods ``` Any idea? More information on masking: http://www.ats.ucla.edu/stat/r/faq/referencing_objects.htm http://www.sfu.ca/~sweldon/howTo/guide4.pdf This is what the output doc looks like (minus css): ![image](https://cloud.githubusercontent.com/assets/8969467/11229714/2946e5de-8d4d-11e5-94b0-dda9696b6fdd.png) Author: felixcheung <felixcheung_m@hotmail.com> Closes #9785 from felixcheung/rmasked.
* [SPARK-11842][ML] Small cleanups to existing Readers and WritersJoseph K. Bradley2015-11-1810-25/+38
| | | | | | | | | | | | | | | | | Updates: * Add repartition(1) to save() methods' saving of data for LogisticRegressionModel, LinearRegressionModel. * Strengthen privacy to class and companion object for Writers and Readers * Change LogisticRegressionSuite read/write test to fit intercept * Add Since versions for read/write methods in Pipeline, LogisticRegression * Switch from hand-written class names in Readers to using getClass CC: mengxr CC: yanboliang Would you mind taking a look at this PR? mengxr might not be able to soon. Thank you! Author: Joseph K. Bradley <joseph@databricks.com> Closes #9829 from jkbradley/ml-io-cleanups.
* [SPARK-11614][SQL] serde parameters should be set only when all params are readynavis.ryu2015-11-181-1/+2
| | | | | | | | | | see HIVE-7975 and HIVE-12373 With changed semantic of setters in thrift objects in hive, setter should be called only after all parameters are set. It's not problem of current state but will be a problem in some day. Author: navis.ryu <navis@apache.org> Closes #9580 from navis/SPARK-11614.
* [SPARK-11816][ML] fix some style issue in ML/MLlib examplesYuhao Yang2015-11-185-3/+5
| | | | | | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11816 Currently I only fixed some obvious comments issue like // scalastyle:off println on the bottom. Yet the style in examples is not quite consistent, like only half of the examples are with // Example usage: ./bin/run-example mllib.FPGrowthExample \, Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9808 from hhbyyh/exampleStyle.
* Revert "[SPARK-11544][SQL] sqlContext doesn't use PathFilter"Yin Huai2015-11-182-54/+7
| | | | This reverts commit 54db79702513e11335c33bcf3a03c59e965e6f16.
* [SPARK-11787][SQL] Improve Parquet scan performance when using flat schemas.Nong Li2015-11-188-43/+944
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch adds an alternate to the Parquet RecordReader from the parquet-mr project that is much faster for flat schemas. Instead of using the general converter mechanism from parquet-mr, this directly uses the lower level APIs from parquet-columnar and a customer RecordReader that directly assembles into UnsafeRows. This is optionally disabled and only used for supported schemas. Using the tpcds store sales table and doing a sum of increasingly more columns, the results are: For 1 Column: Before: 11.3M rows/second After: 18.2M rows/second For 2 Columns: Before: 7.2M rows/second After: 11.2M rows/second For 5 Columns: Before: 2.9M rows/second After: 4.5M rows/second Author: Nong Li <nong@databricks.com> Closes #9774 from nongli/parquet.
* [SPARK-11833][SQL] Add Java tests for Kryo/Java Dataset encodersReynold Xin2015-11-184-40/+166
| | | | | | | | Also added some nicer error messages for incompatible types (private types and primitive types) for Kryo/Java encoder. Author: Reynold Xin <rxin@databricks.com> Closes #9823 from rxin/SPARK-11833.
* [SPARK-11839][ML] refactor save/write traitsXiangrui Meng2015-11-1827-321/+190
| | | | | | | | | | | | * add "ML" prefix to reader/writer/readable/writable to avoid name collision with java.util.* * define `DefaultParamsReadable/Writable` and use them to save some code * use `super.load` instead so people can jump directly to the doc of `Readable.load`, which documents the Java compatibility issues jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #9827 from mengxr/SPARK-11839.
* [SPARK-11636][SQL] Support classes defined in the REPL with EncodersMichael Armbrust2015-11-1817-82/+193
| | | | | | | | | | | | Before this PR there were two things that would blow up if you called `df.as[MyClass]` if `MyClass` was defined in the REPL: - [x] Because `classForName` doesn't work on the munged names returned by `tpe.erasure.typeSymbol.asClass.fullName` - [x] Because we don't have anything to pass into the constructor for the `$outer` pointer. Note that this PR is just adding the infrastructure for working with inner classes in encoder and is not yet sufficient to make them work in the REPL. Currently, the implementation show in https://github.com/marmbrus/spark/commit/95cec7d413b930b36420724fafd829bef8c732ab is causing a bug that breaks code gen due to some interaction between janino and the `ExecutorClassLoader`. This will be addressed in a follow-up PR. Author: Michael Armbrust <michael@databricks.com> Closes #9602 from marmbrus/dataset-replClasses.
* [SPARK-11791] Fix flaky test in BatchedWriteAheadLogSuiteBurak Yavuz2015-11-181-4/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | stack trace of failure: ``` org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 62 times over 1.006322071 seconds. Last failure message: Argument(s) are different! Wanted: writeAheadLog.write( java.nio.HeapByteBuffer[pos=0 lim=124 cap=124], 10 ); -> at org.apache.spark.streaming.util.BatchedWriteAheadLogSuite$$anonfun$23$$anonfun$apply$mcV$sp$15.apply(WriteAheadLogSuite.scala:518) Actual invocation has different arguments: writeAheadLog.write( java.nio.HeapByteBuffer[pos=0 lim=124 cap=124], 10 ); -> at org.apache.spark.streaming.util.WriteAheadLogSuite$BlockingWriteAheadLog.write(WriteAheadLogSuite.scala:756) ``` I believe the issue was that due to a race condition, the ordering of the events could be messed up in the final ByteBuffer, therefore the comparison fails. By adding eventually between the requests, we make sure the ordering is preserved. Note that in real life situations, the ordering across threads will not matter. Another solution would be to implement a custom mockito matcher that sorts and then compares the results, but that kind of sounds like overkill to me. Let me know what you think tdas zsxwing Author: Burak Yavuz <brkyvz@gmail.com> Closes #9790 from brkyvz/fix-flaky-2.
* [SPARK-11814][STREAMING] Add better default checkpoint durationTathagata Das2015-11-182-1/+56
| | | | | | | | | DStream checkpoint interval is by default set at max(10 second, batch interval). That's bad for large batch intervals where the checkpoint interval = batch interval, and RDDs get checkpointed every batch. This PR is to set the checkpoint interval of trackStateByKey to 10 * batch duration. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9805 from tdas/SPARK-11814.
* [SPARK-11495] Fix potential socket / file handle leaks that were found via ↵Josh Rosen2015-11-186-53/+90
| | | | | | | | | | static analysis The HP Fortify Opens Source Review team (https://www.hpfod.com/open-source-review-project) reported a handful of potential resource leaks that were discovered using their static analysis tool. We should fix the issues identified by their scan. Author: Josh Rosen <joshrosen@databricks.com> Closes #9455 from JoshRosen/fix-potential-resource-leaks.
* [SPARK-10930] History "Stages" page "duration" can be confusingDerek Dagit2015-11-181-3/+16
| | | | | | Author: Derek Dagit <derekd@yahoo-inc.com> Closes #9051 from d2r/spark-10930-ui-max-task-dur.
* [SPARK-11649] Properly set Akka frame size in SparkListenerSuite testJosh Rosen2015-11-181-2/+3
| | | | | | | | | | SparkListenerSuite's _"onTaskGettingResult() called when result fetched remotely"_ test was extremely slow (1 to 4 minutes to run) and recently became extremely flaky, frequently failing with OutOfMemoryError. The root cause was the fact that this was using `System.setProperty` to set the Akka frame size, which was not actually modifying the frame size. As a result, this test would allocate much more data than necessary. The fix here is to simply use SparkConf in order to configure the frame size. Author: Josh Rosen <joshrosen@databricks.com> Closes #9822 from JoshRosen/SPARK-11649.
* [SPARK-6787][ML] add read/write to estimators under ml.feature (1)Xiangrui Meng2015-11-1810-47/+467
| | | | | | | | | | | | | | | | | | Add read/write support to the following estimators under spark.ml: * CountVectorizer * IDF * MinMaxScaler * StandardScaler (a little awkward because we store some params in spark.mllib model) * StringIndexer Added some necessary method for read/write. Maybe we should add `private[ml] trait DefaultParamsReadable` and `DefaultParamsWritable` to save some boilerplate code, though we still need to override `load` for Java compatibility. jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #9798 from mengxr/SPARK-6787.
* [SPARK-11810][SQL] Java-based encoder for opaque types in Datasets.Reynold Xin2015-11-184-41/+130
| | | | | | | | This patch refactors the existing Kryo encoder expressions and adds support for Java serialization. Author: Reynold Xin <rxin@databricks.com> Closes #9802 from rxin/SPARK-11810.
* [SPARK-11544][SQL] sqlContext doesn't use PathFilterDilip Biswal2015-11-182-7/+54
| | | | | | | | Apply the user supplied pathfilter while retrieving the files from fs. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #9652 from dilipbiswal/spark-11544.
* [SPARK-11820][ML][PYSPARK] PySpark LiR & LoR should support weightColYanbo Liang2015-11-182-16/+17
| | | | | | | | [SPARK-7685](https://issues.apache.org/jira/browse/SPARK-7685) and [SPARK-9642](https://issues.apache.org/jira/browse/SPARK-9642) have already supported setting weight column for ```LogisticRegression``` and ```LinearRegression```. It's a very important feature, PySpark should also support. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9811 from yanboliang/spark-11820.
* [SPARK-11684][R][ML][DOC] Update SparkR glm API doc, user guide and example ↵Yanbo Liang2015-11-183-11/+60
| | | | | | | | | | | | | | | | | codes This PR includes: * Update SparkR:::glm, SparkR:::summary API docs. * Update SparkR machine learning user guide and example codes to show: * supporting feature interaction in R formula. * summary for gaussian GLM model. * coefficients for binomial GLM model. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9727 from yanboliang/spark-11684.
* [SPARK-11813][MLLIB] Avoid serialization of vocab in Word2VecYuhao Yang2015-11-181-2/+2
| | | | | | | | | | | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11813 I found the problem during training a large corpus. Avoid serialization of vocab in Word2Vec has 2 benefits. 1. Performance improvement for less serialization. 2. Increase the capacity of Word2Vec a lot. Currently in the fit of word2vec, the closure mainly includes serialization of Word2Vec and 2 global table. the main part of Word2vec is the vocab of size: vocab * 40 * 2 * 4 = 320 vocab 2 global table: vocab * vectorSize * 8. If vectorSize = 20, that's 160 vocab. Their sum cannot exceed Int.max due to the restriction of ByteArrayOutputStream. In any case, avoiding serialization of vocab helps decrease the size of the closure serialization, especially when vectorSize is small, thus to allow larger vocabulary. Actually there's another possible fix, make local copy of fields to avoid including Word2Vec in the closure. Let me know if that's preferred. Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9803 from hhbyyh/w2vVocab.
* [SPARK-6789][ML] Add Readable, Writable support for spark.ml ALS, ALSModelJoseph K. Bradley2015-11-183-17/+150
| | | | | | | | | | Also modifies DefaultParamsWriter.saveMetadata to take optional extra metadata. CC: mengxr yanboliang Author: Joseph K. Bradley <joseph@databricks.com> Closes #9786 from jkbradley/als-io.
* [SPARK-6790][ML] Add spark.ml LinearRegression import/exportWenjian Huang2015-11-182-5/+106
| | | | | | | | | | | | | This replaces [https://github.com/apache/spark/pull/9656] with updates. fayeshine should be the main author when this PR is committed. CC: mengxr fayeshine Author: Wenjian Huang <nextrush@163.com> Author: Joseph K. Bradley <joseph@databricks.com> Closes #9814 from jkbradley/fayeshine-patch-6790.
* [SPARK-11720][SQL][ML] Handle edge cases when count = 0 or 1 for Stats functionJihongMa2015-11-188-25/+53
| | | | | | | | return Double.NaN for mean/average when count == 0 for all numeric types that is converted to Double, Decimal type continue to return null. Author: JihongMa <linlin200605@gmail.com> Closes #9705 from JihongMA/SPARK-11720.
* [SPARK-10745][CORE] Separate configs between shuffle and RPCShixiong Zhu2015-11-1823-50/+84
| | | | | | | | | | [SPARK-6028](https://issues.apache.org/jira/browse/SPARK-6028) uses network module to implement RPC. However, there are some configurations named with `spark.shuffle` prefix in the network module. This PR refactors them to make sure the user can control them in shuffle and RPC separately. The user can use `spark.rpc.*` to set the configuration for netty RPC. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9481 from zsxwing/SPARK-10745.
* [SPARK-11809] Switch the default Mesos mode to coarse-grained modeReynold Xin2015-11-183-12/+19
| | | | | | | | Based on my conversions with people, I believe the consensus is that the coarse-grained mode is more stable and easier to reason about. It is best to use that as the default rather than the more flaky fine-grained mode. Author: Reynold Xin <rxin@databricks.com> Closes #9795 from rxin/SPARK-11809.
* [SPARK-4557][STREAMING] Spark Streaming foreachRDD Java API method should ↵Bryan Cutler2015-11-185-2/+120
| | | | | | | | | | accept a VoidFunction<...> Currently streaming foreachRDD Java API uses a function prototype requiring a return value of null. This PR deprecates the old method and uses VoidFunction to allow for more concise declaration. Also added VoidFunction2 to Java API in order to use in Streaming methods. Unit test is added for using foreachRDD with VoidFunction, and changes have been tested with Java 7 and Java 8 using lambdas. Author: Bryan Cutler <bjcutler@us.ibm.com> Closes #9488 from BryanCutler/foreachRDD-VoidFunction-SPARK-4557.
* [SPARK-11739][SQL] clear the instantiated SQLContextDavies Liu2015-11-183-10/+14
| | | | | | | | Currently, if the first SQLContext is not removed after stopping SparkContext, a SQLContext could set there forever. This patch make this more robust. Author: Davies Liu <davies@databricks.com> Closes #9706 from davies/clear_context.
* [SPARK-11792] [SQL] [FOLLOW-UP] Change SizeEstimation to KnownSizeEstimation ↵Yin Huai2015-11-183-30/+26
| | | | | | | | | | | | | | | | | and make estimatedSize return Long instead of Option[Long] https://issues.apache.org/jira/browse/SPARK-11792 The main changes include: * Renaming `SizeEstimation` to `KnownSizeEstimation`. Hopefully this new name has more information. * Making `estimatedSize` return `Long` instead of `Option[Long]`. * In `UnsaveHashedRelation`, `estimatedSize` will delegate the work to `SizeEstimator` if we have not created a `BytesToBytesMap`. Since we will put `UnsaveHashedRelation` to `BlockManager`, it is generally good to let it provide a more accurate size estimation. Also, if we do not put `BytesToBytesMap` directly into `BlockerManager`, I feel it is not really necessary to make `BytesToBytesMap` extends `KnownSizeEstimation`. Author: Yin Huai <yhuai@databricks.com> Closes #9813 from yhuai/SPARK-11792-followup.
* [MINOR][BUILD] Ignore ensime cacheJakob Odersky2015-11-181-0/+1
| | | | | | | | Using ENSIME, I often have `.ensime_cache` polluting my source tree. This PR simply adds the cache directory to `.gitignore` Author: Jakob Odersky <jodersky@gmail.com> Closes #9708 from jodersky/master.
* [SPARK-11795][SQL] combine grouping attributes into a single NamedExpressionWenchen Fan2015-11-182-5/+9
| | | | | | | | | we use `ExpressionEncoder.tuple` to build the result encoder, which assumes the input encoder should point to a struct type field if it’s non-flat. However, our keyEncoder always point to a flat field/fields: `groupingAttributes`, we should combine them into a single `NamedExpression`. Author: Wenchen Fan <wenchen@databricks.com> Closes #9792 from cloud-fan/agg.
* [SPARK-11725][SQL] correctly handle null inputs for UDFWenchen Fan2015-11-186-1/+121
| | | | | | | | If user use primitive parameters in UDF, there is no way for him to do the null-check for primitive inputs, so we are assuming the primitive input is null-propagatable for this case and return null if the input is null. Author: Wenchen Fan <wenchen@databricks.com> Closes #9770 from cloud-fan/udf.
* [SPARK-11803][SQL] fix Dataset self-joinWenchen Fan2015-11-182-9/+13
| | | | | | | | When we resolve the join operator, we may change the output of right side if self-join is detected. So in `Dataset.joinWith`, we should resolve the join operator first, and then get the left output and right output from it, instead of using `left.output` and `right.output` directly. Author: Wenchen Fan <wenchen@databricks.com> Closes #9806 from cloud-fan/self-join.
* [SPARK-11195][CORE] Use correct classloader for TaskResultGetterHurshal Patel2015-11-183-8/+72
| | | | | | | | | | | | | | | Make sure we are using the context classloader when deserializing failed TaskResults instead of the Spark classloader. The issue is that `enqueueFailedTask` was using the incorrect classloader which results in `ClassNotFoundException`. Adds a test in TaskResultGetterSuite that compiles a custom exception, throws it on the executor, and asserts that Spark handles the TaskResult deserialization instead of returning `UnknownReason`. See #9367 for previous comments See SPARK-11195 for a full repro Author: Hurshal Patel <hpatel516@gmail.com> Closes #9779 from choochootrain/spark-11195-master.
* [SPARK-11773][SPARKR] Implement collection functions in SparkR.Sun Rui2015-11-186-35/+100
| | | | | | Author: Sun Rui <rui.sun@intel.com> Closes #9764 from sun-rui/SPARK-11773.
* [SPARK-11281][SPARKR] Add tests covering the issue.zero3232015-11-181-3/+7
| | | | | | | | The goal of this PR is to add tests covering the issue to ensure that is was resolved by [SPARK-11086](https://issues.apache.org/jira/browse/SPARK-11086). Author: zero323 <matthew.szymkiewicz@gmail.com> Closes #9743 from zero323/SPARK-11281-tests.