aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-11636][SQL] Support classes defined in the REPL with EncodersMichael Armbrust2015-11-201-2/+2
| | | | | | | | | #theScaryParts (i.e. changes to the repl, executor classloaders and codegen)... Author: Michael Armbrust <michael@databricks.com> Author: Yin Huai <yhuai@databricks.com> Closes #9825 from marmbrus/dataset-replClasses2.
* [SPARK-11716][SQL] UDFRegistration just drops the input type when ↵Jean-Baptiste Onofré2015-11-202-24/+39
| | | | | | | | | | | | | | | | | re-creating the UserDefinedFunction https://issues.apache.org/jira/browse/SPARK-11716 This is one is #9739 and a regression test. When commit it, please make sure the author is jbonofre. You can find the original PR at https://github.com/apache/spark/pull/9739 closes #9739 Author: Jean-Baptiste Onofré <jbonofre@apache.org> Author: Yin Huai <yhuai@databricks.com> Closes #9868 from yhuai/SPARK-11716.
* [SPARK-11724][SQL] Change casting between int and timestamp to consistently ↵Nong Li2015-11-208-25/+39
| | | | | | | | | | | | treat int in seconds. Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454 Author: Nong Li <nong@databricks.com> Author: Nong Li <nongli@gmail.com> Author: Yin Huai <yhuai@databricks.com> Closes #9685 from nongli/spark-11724.
* [SPARK-11819][SQL] nice error message for missing encoderWenchen Fan2015-11-202-23/+129
| | | | | | | | | | | | | | | | before this PR, when users try to get an encoder for an un-supported class, they will only get a very simple error message like `Encoder for type xxx is not supported`. After this PR, the error message become more friendly, for example: ``` No Encoder found for abc.xyz.NonEncodable - array element class: "abc.xyz.NonEncodable" - field (class: "scala.Array", name: "arrayField") - root class: "abc.xyz.AnotherClass" ``` Author: Wenchen Fan <wenchen@databricks.com> Closes #9810 from cloud-fan/error-message.
* [SPARK-11817][SQL] Truncating the fractional seconds to prevent inserting a NULLLiang-Chi Hsieh2015-11-202-0/+13
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-11817 Instead of return None, we should truncate the fractional seconds to prevent inserting NULL. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #9834 from viirya/truncate-fractional-sec.
* [SPARK-11876][SQL] Support printSchema in DataSet APIgatorsmile2015-11-202-9/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DataSet APIs look great! However, I am lost when doing multiple level joins. For example, ``` val ds1 = Seq(("a", 1), ("b", 2)).toDS().as("a") val ds2 = Seq(("a", 1), ("b", 2)).toDS().as("b") val ds3 = Seq(("a", 1), ("b", 2)).toDS().as("c") ds1.joinWith(ds2, $"a._2" === $"b._2").as("ab").joinWith(ds3, $"ab._1._2" === $"c._2").printSchema() ``` The printed schema is like ``` root |-- _1: struct (nullable = true) | |-- _1: struct (nullable = true) | | |-- _1: string (nullable = true) | | |-- _2: integer (nullable = true) | |-- _2: struct (nullable = true) | | |-- _1: string (nullable = true) | | |-- _2: integer (nullable = true) |-- _2: struct (nullable = true) | |-- _1: string (nullable = true) | |-- _2: integer (nullable = true) ``` Personally, I think we need the printSchema function. Sometimes, I do not know how to specify the column, especially when their data types are mixed. For example, if I want to write the following select for the above multi-level join, I have to know the schema: ``` newDS.select(expr("_1._2._2 + 1").as[Int]).collect() ``` marmbrus rxin cloud-fan Do you have the same feeling? Author: gatorsmile <gatorsmile@gmail.com> Closes #9855 from gatorsmile/printSchemaDataSet.
* [SPARK-11877] Prevent agg. fallback conf. from leaking across test suitesJosh Rosen2015-11-201-23/+21
| | | | | | | | | | This patch fixes an issue where the `spark.sql.TungstenAggregate.testFallbackStartsAt` SQLConf setting was not properly reset / cleared at the end of `TungstenAggregationQueryWithControlledFallbackSuite`. This ended up causing test failures in HiveCompatibilitySuite in Maven builds by causing spilling to occur way too frequently. This configuration leak was inadvertently introduced during test cleanup in #9618. Author: Josh Rosen <joshrosen@databricks.com> Closes #9857 from JoshRosen/clear-fallback-prop-in-test-teardown.
* [SPARK-11544][SQL][TEST-HADOOP1.0] sqlContext doesn't use PathFilterDilip Biswal2015-11-192-7/+59
| | | | | | | | Apply the user supplied pathfilter while retrieving the files from fs. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #9830 from dilipbiswal/spark-11544.
* [SPARK-11864][SQL] Improve performance of max/minDavies Liu2015-11-195-25/+45
| | | | | | | | | | | | | | | | This PR has the following optimization: 1) The greatest/least already does the null-check, so the `If` and `IsNull` are not necessary. 2) In greatest/least, it should initialize the result using the first child (removing one block). 3) For primitive types, the generated greater expression is too complicated (`a > b ? 1 : (a < b) ? -1 : 0) > 0`), should be as simple as `a > b` Combine these optimization, this could improve the performance of `ss_max` query by 30%. Author: Davies Liu <davies@databricks.com> Closes #9846 from davies/improve_max.
* [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-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-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-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-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.
* 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-187-36/+910
| | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-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-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-11720][SQL][ML] Handle edge cases when count = 0 or 1 for Stats functionJihongMa2015-11-187-24/+52
| | | | | | | | 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-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-181-4/+8
| | | | | | | | | | | | | | | | | 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.
* [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-10946][SQL] JDBC - Use Statement.executeUpdate instead of ↵somideshmukh2015-11-182-2/+2
| | | | | | | | | | PreparedStatement.executeUpdate for DDLs New changes with JDBCRDD Author: somideshmukh <somilde@us.ibm.com> Closes #9733 from somideshmukh/SomilBranch-1.1.
* [SPARK-11792][SQL] SizeEstimator cannot provide a good size estimation of ↵Yin Huai2015-11-181-2/+8
| | | | | | | | | | | | UnsafeHashedRelations https://issues.apache.org/jira/browse/SPARK-11792 Right now, SizeEstimator will "think" a small UnsafeHashedRelation is several GBs. Author: Yin Huai <yhuai@databricks.com> Closes #9788 from yhuai/SPARK-11792.
* [SPARK-11802][SQL] Kryo-based encoder for opaque types in DatasetsReynold Xin2015-11-188-23/+178
| | | | | | | | I also found a bug with self-joins returning incorrect results in the Dataset API. Two test cases attached and filed SPARK-11803. Author: Reynold Xin <rxin@databricks.com> Closes #9789 from rxin/SPARK-11802.
* [SPARK-11643] [SQL] parse year with leading zeroDavies Liu2015-11-172-5/+32
| | | | | | | | Support the years between 0 <= year < 1000 Author: Davies Liu <davies@databricks.com> Closes #9701 from davies/leading_zero.
* [SPARK-11797][SQL] collect, first, and take should use encoders for ↵Reynold Xin2015-11-172-6/+41
| | | | | | | | | | serialization They were previously using Spark's default serializer for serialization. Author: Reynold Xin <rxin@databricks.com> Closes #9787 from rxin/SPARK-11797.
* [SPARK-11793][SQL] Dataset should set the resolved encoders internally for maps.Reynold Xin2015-11-172-1/+13
| | | | | | | | I also wrote a test case -- but unfortunately the test case is not working due to SPARK-11795. Author: Reynold Xin <rxin@databricks.com> Closes #9784 from rxin/SPARK-11503.
* [SPARK-11767] [SQL] limit the size of caced batchDavies Liu2015-11-173-4/+16
| | | | | | | | | | | | | | Currently the size of cached batch in only controlled by `batchSize` (default value is 10000), which does not work well with the size of serialized columns (for example, complex types). The memory used to build the batch is not accounted, it's easy to OOM (especially after unified memory management). This PR introduce a hard limit as 4M for total columns (up to 50 columns of uncompressed primitive columns). This also change the way to grow buffer, double it each time, then trim it once finished. cc liancheng Author: Davies Liu <davies@databricks.com> Closes #9760 from davies/cache_limit.
* [SPARK-10186][SQL] support postgre array type in JDBCRDDWenchen Fan2015-11-174-69/+129
| | | | | | | | | | | | | | Add ARRAY support to `PostgresDialect`. Nested ARRAY is not allowed for now because it's hard to get the array dimension info. See http://stackoverflow.com/questions/16619113/how-to-get-array-base-type-in-postgres-via-jdbc Thanks for the initial work from mariusvniekerk ! Close https://github.com/apache/spark/pull/9137 Author: Wenchen Fan <wenchen@databricks.com> Closes #9662 from cloud-fan/postgre.
* [SPARK-8658][SQL][FOLLOW-UP] AttributeReference's equals method compares all ↵gatorsmile2015-11-172-2/+9
| | | | | | | | | | | | | | the members Based on the comment of cloud-fan in https://github.com/apache/spark/pull/9216, update the AttributeReference's hashCode function by including the hashCode of the other attributes including name, nullable and qualifiers. Here, I am not 100% sure if we should include name in the hashCode calculation, since the original hashCode calculation does not include it. marmbrus cloud-fan Please review if the changes are good. Author: gatorsmile <gatorsmile@gmail.com> Closes #9761 from gatorsmile/hashCodeNamedExpression.
* [SPARK-11089][SQL] Adds option for disabling multi-session in Thrift serverCheng Lian2015-11-173-2/+58
| | | | | | | | | | This PR adds a new option `spark.sql.hive.thriftServer.singleSession` for disabling multi-session support in the Thrift server. Note that this option is added as a Spark configuration (retrieved from `SparkConf`) rather than Spark SQL configuration (retrieved from `SQLConf`). This is because all SQL configurations are session-ized. Since multi-session support is by default on, no JDBC connection can modify global configurations like the newly added one. Author: Cheng Lian <lian@databricks.com> Closes #9740 from liancheng/spark-11089.single-session-option.
* [SPARK-11679][SQL] Invoking method " apply(fields: ↵mayuanwen2015-11-172-1/+15
| | | | | | | | | | | java.util.List[StructField])" in "StructType" gets ClassCastException In the previous method, fields.toArray will cast java.util.List[StructField] into Array[Object] which can not cast into Array[StructField], thus when invoking this method will throw "java.lang.ClassCastException: [Ljava.lang.Object; cannot be cast to [Lorg.apache.spark.sql.types.StructField;" I directly cast java.util.List[StructField] into Array[StructField] in this patch. Author: mayuanwen <mayuanwen@qiyi.com> Closes #9649 from jackieMaKing/Spark-11679.
* [SPARK-11191][SQL][FOLLOW-UP] Cleans up unnecessary anonymous ↵Cheng Lian2015-11-172-11/+6
| | | | | | | | | | HiveFunctionRegistry According to discussion in PR #9664, the anonymous `HiveFunctionRegistry` in `HiveContext` can be removed now. Author: Cheng Lian <lian@databricks.com> Closes #9737 from liancheng/spark-11191.follow-up.
* [MINOR] [SQL] Fix randomly generated ArrayData in RowEncoderSuiteLiang-Chi Hsieh2015-11-161-1/+8
| | | | | | | | The randomly generated ArrayData used for the UDT `ExamplePoint` in `RowEncoderSuite` sometimes doesn't have enough elements. In this case, this test will fail. This patch is to fix it. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #9757 from viirya/fix-randomgenerated-udt.
* [SPARK-11447][SQL] change NullType to StringType during binaryComparison ↵Kevin Yu2015-11-162-0/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | between NullType and StringType During executing PromoteStrings rule, if one side of binaryComparison is StringType and the other side is not StringType, the current code will promote(cast) the StringType to DoubleType, and if the StringType doesn't contain the numbers, it will get null value. So if it is doing <=> (NULL-safe equal) with Null, it will not filter anything, caused the problem reported by this jira. I proposal to the changes through this PR, can you review my code changes ? This problem only happen for <=>, other operators works fine. scala> val filteredDF = df.filter(df("column") > (new Column(Literal(null)))) filteredDF: org.apache.spark.sql.DataFrame = [column: string] scala> filteredDF.show +------+ |column| +------+ +------+ scala> val filteredDF = df.filter(df("column") === (new Column(Literal(null)))) filteredDF: org.apache.spark.sql.DataFrame = [column: string] scala> filteredDF.show +------+ |column| +------+ +------+ scala> df.registerTempTable("DF") scala> sqlContext.sql("select * from DF where 'column' = NULL") res27: org.apache.spark.sql.DataFrame = [column: string] scala> res27.show +------+ |column| +------+ +------+ Author: Kevin Yu <qyu@us.ibm.com> Closes #9720 from kevinyu98/working_on_spark-11447.
* [SPARK-11694][FOLLOW-UP] Clean up imports, use a common function for ↵hyukjinkwon2015-11-172-27/+15
| | | | | | | | | | | | metadata and add a test for FIXED_LEN_BYTE_ARRAY As discussed https://github.com/apache/spark/pull/9660 https://github.com/apache/spark/pull/9060, I cleaned up unused imports, added a test for fixed-length byte array and used a common function for writing metadata for Parquet. For the test for fixed-length byte array, I have tested and checked the encoding types with [parquet-tools](https://github.com/Parquet/parquet-mr/tree/master/parquet-tools). Author: hyukjinkwon <gurwls223@gmail.com> Closes #9754 from HyukjinKwon/SPARK-11694-followup.
* [SPARK-11768][SPARK-9196][SQL] Support now function in SQL (alias for ↵Reynold Xin2015-11-162-6/+13
| | | | | | | | | | | | current_timestamp). This patch adds an alias for current_timestamp (now function). Also fixes SPARK-9196 to re-enable the test case for current_timestamp. Author: Reynold Xin <rxin@databricks.com> Closes #9753 from rxin/SPARK-11768.
* [SPARK-11625][SQL] add java test for typed aggregateWenchen Fan2015-11-164-8/+91
| | | | | | Author: Wenchen Fan <wenchen@databricks.com> Closes #9591 from cloud-fan/agg-test.
* [SPARK-8658][SQL] AttributeReference's equals method compares all the membersgatorsmile2015-11-163-12/+14
| | | | | | | | This fix is to change the equals method to check all of the specified fields for equality of AttributeReference. Author: gatorsmile <gatorsmile@gmail.com> Closes #9216 from gatorsmile/namedExpressEqual.
* [SPARK-11553][SQL] Primitive Row accessors should not convert null to ↵Bartlomiej Alberski2015-11-163-23/+65
| | | | | | | | | | default value Invocation of getters for type extending AnyVal returns default value (if field value is null) instead of throwing NPE. Please check comments for SPARK-11553 issue for more details. Author: Bartlomiej Alberski <bartlomiej.alberski@allegrogroup.com> Closes #9642 from alberskib/bugfix/SPARK-11553.
* [SPARK-11390][SQL] Query plan with/without filterPushdown indistinguishableZee Chen2015-11-163-4/+22
| | | | | | | | | | …ishable Propagate pushed filters to PhyicalRDD in DataSourceStrategy.apply Author: Zee Chen <zeechen@us.ibm.com> Closes #9679 from zeocio/spark-11390.