- May 22, 2017
-
-
Michal Senkyr authored
## What changes were proposed in this pull request? Replaced specific sequence encoders with generic sequence encoder to enable nesting of sequences. Does not add support for nested arrays as that cannot be solved in this way. ## How was this patch tested? ```bash build/mvn -DskipTests clean package && dev/run-tests ``` Additionally in Spark shell: ``` scala> Seq(Seq(Seq(1))).toDS.collect() res0: Array[Seq[Seq[Int]]] = Array(List(List(1))) ``` Author: Michal Senkyr <mike.senkyr@gmail.com> Closes #18011 from michalsenkyr/dataset-seq-nested.
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR improves the implementation of `ColumnStats` by using the following appoaches. 1. Declare subclasses of `ColumnStats` as `final` 2. Remove unnecessary call of `row.isNullAt(ordinal)` 3. Remove the dependency on `GenericInternalRow` For 1., this declaration encourages method inlining and other optimizations of JIT compiler For 2., in `gatherStats()`, while previous code in subclasses of `ColumnStats` always calls `row.isNullAt()` twice, the PR just calls `row.isNullAt()` only once. For 3., `collectedStatistics()` returns `Array[Any]` instead of `GenericInternalRow`. This removes the dependency of unnecessary package and reduces the number of allocations of `GenericInternalRow`. In addition to that, in the future, `gatherValueStats()`, which is specialized for each data type, can be effectively called from the generated code without using generic data structure `InternalRow`. ## How was this patch tested? Tested by existing test suite Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #18002 from kiszk/SPARK-20770.
-
caoxuewen authored
## What changes were proposed in this pull request? spark-sql>SELECT ceil(1234567890123456); 1234567890123456 spark-sql>SELECT ceil(12345678901234567); 12345678901234568 spark-sql>SELECT ceil(123456789012345678); 123456789012345680 when the length of the getText is greater than 16. long to double will be precision loss. but mysql handle the value is ok. mysql> SELECT ceil(1234567890123456); +------------------------+ | ceil(1234567890123456) | +------------------------+ | 1234567890123456 | +------------------------+ 1 row in set (0.00 sec) mysql> SELECT ceil(12345678901234567); +-------------------------+ | ceil(12345678901234567) | +-------------------------+ | 12345678901234567 | +-------------------------+ 1 row in set (0.00 sec) mysql> SELECT ceil(123456789012345678); +--------------------------+ | ceil(123456789012345678) | +--------------------------+ | 123456789012345678 | +--------------------------+ 1 row in set (0.00 sec) ## How was this patch tested? Supplement the unit test. Author: caoxuewen <cao.xuewen@zte.com.cn> Closes #18016 from heary-cao/ceil_long.
-
- May 21, 2017
-
-
Tathagata Das authored
[SPARK-20792][SS] Support same timeout operations in mapGroupsWithState function in batch queries as in streaming queries ## What changes were proposed in this pull request? Currently, in the batch queries, timeout is disabled (i.e. GroupStateTimeout.NoTimeout) which means any GroupState.setTimeout*** operation would throw UnsupportedOperationException. This makes it weird when converting a streaming query into a batch query by changing the input DF from streaming to a batch DF. If the timeout was enabled and used, then the batch query will start throwing UnsupportedOperationException. This PR creates the dummy state in batch queries with the provided timeoutConf so that it behaves in the same way. The code has been refactored to make it obvious when the state is being created for a batch query or a streaming query. ## How was this patch tested? Additional tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #18024 from tdas/SPARK-20792.
-
- May 19, 2017
-
-
liuxian authored
## What changes were proposed in this pull request? spark-sql>select month("1582-09-28"); spark-sql>10 For this case, the expected result is 9, but it is 10. spark-sql>select day("1582-04-18"); spark-sql>28 For this case, the expected result is 18, but it is 28. when the date before "1582-10-04", the function of `month` and `day` return the value which is not we expected. ## How was this patch tested? unit tests Author: liuxian <liu.xian3@zte.com.cn> Closes #17997 from 10110346/wip_lx_0516.
-
Yuming Wang authored
## What changes were proposed in this pull request? Add built-in SQL Function - COT. ## How was this patch tested? unit tests Author: Yuming Wang <wgyumg@gmail.com> Closes #17999 from wangyum/SPARK-20751.
-
tpoterba authored
Fix quadratic List indexing in ParquetWriteSupport. I noticed this function while profiling some code with today. It showed up as a significant factor in a table with twenty columns; with hundreds of columns, it could dominate any other function call. ## What changes were proposed in this pull request? The writeFields method iterates from 0 until number of fields, indexing into rootFieldWriters for each element. rootFieldWriters is a List, so indexing is a linear operation. The complexity of the writeFields method is thus quadratic in the number of fields. Solution: explicitly convert rootFieldWriters to Array (implicitly converted to WrappedArray) for constant-time indexing. ## How was this patch tested? This is a one-line change for performance reasons. Author: tpoterba <tpoterba@broadinstitute.org> Author: Tim Poterba <tpoterba@gmail.com> Closes #18005 from tpoterba/tpoterba-patch-1.
-
Ala Luszczak authored
## What changes were proposed in this pull request? GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior. Example of code generated before: ```scala /* 059 */ final UTF8String fieldName = value.getUTF8String(0); /* 060 */ if (value.isNullAt(0)) { /* 061 */ rowWriter1.setNullAt(0); /* 062 */ } else { /* 063 */ rowWriter1.write(0, fieldName); /* 064 */ } ``` Example of code generated now: ```scala /* 060 */ boolean isNull1 = value.isNullAt(0); /* 061 */ UTF8String value1 = isNull1 ? null : value.getUTF8String(0); /* 062 */ if (isNull1) { /* 063 */ rowWriter1.setNullAt(0); /* 064 */ } else { /* 065 */ rowWriter1.write(0, value1); /* 066 */ } ``` ## How was this patch tested? Adds GenerateUnsafeProjectionSuite. Author: Ala Luszczak <ala@databricks.com> Closes #18030 from ala/fix-generate-unsafe-projection.
-
- May 18, 2017
-
-
hyukjinkwon authored
## What changes were proposed in this pull request? This is an alternative workaround by simply avoiding the predicate pushdown for columns having dots in the names. This is an approach different with https://github.com/apache/spark/pull/17680. The downside of this PR is, literally it does not push down filters on the column having dots in Parquet files at all (both no record level and no rowgroup level) whereas the downside of the approach in that PR, it does not use the Parquet's API properly but in a hacky way to support this case. I assume we prefer a safe way here by using the Parquet API properly but this does close that PR as we are basically just avoiding here. This way looks a simple workaround and probably it is fine given the problem looks arguably rather corner cases (although it might end up with reading whole row groups under the hood but either looks not the best). Currently, if there are dots in the column name, predicate pushdown seems being failed in Parquet. **With dots** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("col.dots").write.parquet(path) spark.read.parquet(path).where("`col.dots` IS NOT NULL").show() ``` ``` +--------+ |col.dots| +--------+ +--------+ ``` **Without dots** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("coldots").write.parquet(path) spark.read.parquet(path).where("`coldots` IS NOT NULL").show() ``` ``` +-------+ |coldots| +-------+ | 1| +-------+ ``` **After** ```scala val path = "/tmp/abcde" Seq(Some(1), None).toDF("col.dots").write.parquet(path) spark.read.parquet(path).where("`col.dots` IS NOT NULL").show() ``` ``` +--------+ |col.dots| +--------+ | 1| +--------+ ``` ## How was this patch tested? Unit tests added in `ParquetFilterSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18000 from HyukjinKwon/SPARK-20364-workaround.
-
Xingbo Jiang authored
## What changes were proposed in this pull request? In the previous approach we used `aliasMap` to link an `Attribute` to the expression with potentially the form `f(a, b)`, but we only searched the `expressions` and `children.expressions` for this, which is not enough when an `Alias` may lies deep in the logical plan. In that case, we can't generate the valid equivalent constraint classes and thus we fail at preventing the recursive deductions. We fix this problem by collecting all `Alias`s from the logical plan. ## How was this patch tested? No additional test case is added, but do modified one test case to cover this situation. Author: Xingbo Jiang <xingbo.jiang@databricks.com> Closes #18020 from jiangxb1987/inferConstrants.
-
- May 16, 2017
-
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this: select 1 from (select 1 from onerow t1 LIMIT 1) where t1.c1=1 This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute. The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17935 from viirya/SPARK-20690.
-
Herman van Hovell authored
## What changes were proposed in this pull request? Currently the parser logs the query it is parsing at `info` level. This is too high, this PR lowers the log level to `debug`. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #18006 from hvanhovell/lower_parser_log_level.
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails. This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught. ## How was this patch tested? Add a test suite into `DataFrameSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17087 from kiszk/SPARK-19372.
-
- May 15, 2017
-
-
Takuya UESHIN authored
## What changes were proposed in this pull request? Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck. This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site. This pr makes a cache of the generated TimeZone instances to avoid the synchronization. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@databricks.com> Closes #17933 from ueshin/issues/SPARK-20588.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_". This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`. ## How was this patch tested? Manual. (Run TPCDSQueryBenchmark) Author: Dongjoon Hyun <dongjoon@apache.org> Closes #17977 from dongjoon-hyun/SPARK-20735.
-
Tathagata Das authored
## What changes were proposed in this pull request? Timeout and state data are two independent entities and should be settable independently. Therefore, in the same call of the user-defined function, one should be able to set the timeout before initializing the state and also after removing the state. Whether timeouts can be set or not, should not depend on the current state, and vice versa. However, a limitation of the current implementation is that state cannot be null while timeout is set. This is checked lazily after the function call has completed. ## How was this patch tested? - Updated existing unit tests that test the behavior of GroupState.setTimeout*** wrt to the current state - Added new tests that verify the disallowed cases where state is undefined but timeout is set. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17957 from tdas/SPARK-20717.
-
Tejas Patil authored
## What changes were proposed in this pull request? Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`. What does this PR achieve ? - Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics). - IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees. Changes done in this PR: - Extract table's bucketing information in `HiveClientImpl` - While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object - `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false` Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit# ## How was this patch tested? - Added test for creating bucketed and sorted table. - Added test to ensure that INSERTs fail if strict bucket / sort is enforced - Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced - Added test to validate that bucketing information shows up in output of DESC FORMATTED - Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables Author: Tejas Patil <tejasp@fb.com> Closes #17644 from tejasapatil/SPARK-17729_create_bucketed_table.
-
Tathagata Das authored
## What changes were proposed in this pull request? StateStore.abort() should do a best effort attempt to clean up temporary resources. It should not throw errors, especially because its called in a TaskCompletionListener, because this error could hide previous real errors in the task. ## How was this patch tested? No unit test. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17958 from tdas/SPARK-20716.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions. ## How was this patch tested? Added tests in `CombineConcatSuite` and `SQLQueryTestSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17970 from maropu/SPARK-20730.
-
- May 13, 2017
-
-
Wenchen Fan authored
## What changes were proposed in this pull request? For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them. ## How was this patch tested? a new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17964 from cloud-fan/tmp.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR is based on https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772 This avoids additional conversion try with `toFloat` and `toDouble`. For avoiding additional conversions, please refer the codes below: **Before** ```scala scala> import org.apache.spark.sql.types._ import org.apache.spark.sql.types._ scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show() 17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2) java.lang.NumberFormatException: For input string: "nan" ... ``` **After** ```scala scala> import org.apache.spark.sql.types._ import org.apache.spark.sql.types._ scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show() 17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.RuntimeException: Cannot parse nan as DoubleType. ... ``` ## How was this patch tested? Unit tests added in `JsonSuite`. Closes #16199 Author: hyukjinkwon <gurwls223@gmail.com> Author: Nathan Howell <nhowell@godaddy.com> Closes #17956 from HyukjinKwon/SPARK-18772.
-
- May 12, 2017
-
-
Xiao Li authored
### What changes were proposed in this pull request? `LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser. ### How was this patch tested? Added a test case Author: Xiao Li <gatorsmile@gmail.com> Closes #17960 from gatorsmile/LimitAll.
-
zuotingbing authored
[SPARK-20594][SQL] The staging directory should be a child directory starts with "." to avoid being deleted if we set hive.exec.stagingdir under the table directory. JIRA Issue: https://issues.apache.org/jira/browse/SPARK-20594 ## What changes were proposed in this pull request? The staging directory should be a child directory starts with "." to avoid being deleted before moving staging directory to table directory if we set hive.exec.stagingdir under the table directory. ## How was this patch tested? Added unit tests Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #17858 from zuotingbing/spark-stagingdir.
-
Tathagata Das authored
[SPARK-20714][SS] Fix match error when watermark is set with timeout = no timeout / processing timeout ## What changes were proposed in this pull request? When watermark is set, and timeout conf is NoTimeout or ProcessingTimeTimeout (both do not need the watermark), the query fails at runtime with the following exception. ``` MatchException: Some(org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate1a9b798e) (of class scala.Some) org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:120) org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:116) org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:70) org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:65) org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:64) ``` The match did not correctly handle cases where watermark was defined by the timeout was different from EventTimeTimeout. ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17954 from tdas/SPARK-20714.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL. ## How was this patch tested? Added tests in `SparkSqlParserSuite` Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17711 from maropu/SPARK-19951.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr added `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191). ## How was this patch tested? Added tests in `SQLQueryTestSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17948 from maropu/SPARK-20710.
-
wangzhenhua authored
## What changes were proposed in this pull request? Fix canonicalization for different filter orders in `HiveTableScanExec`. ## How was this patch tested? Added a new test case. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17962 from wzhfy/canonicalizeHiveTableScanExec.
-
Ryan Blue authored
## What changes were proposed in this pull request? This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters. ## How was this patch tested? This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully. Author: Ryan Blue <blue@apache.org> Closes #15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution.
-
Sean Owen authored
## What changes were proposed in this pull request? Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17949 from srowen/SPARK-20554.
-
hyukjinkwon authored
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement ## What changes were proposed in this pull request? This PR proposes three things as below: - Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`). - Support single argument for `to_timestamp` similarly with APIs in other languages. For example, the one below works ``` import org.apache.spark.sql.functions._ Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show() ``` prints ``` +----------------------------------------+ |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')| +----------------------------------------+ | 2016-12-31 00:12:00| +----------------------------------------+ ``` whereas this does not work in SQL. **Before** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7 ``` **After** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 ``` - Related document improvement for SQL function descriptions and other API descriptions accordingly. **Before** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input. Extended Usage: Examples: > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00.0 ``` **After** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_date('2009-07-30 04:17:52'); 2009-07-30 > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00 ``` ## How was this patch tested? Added tests in `datetime.sql`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17901 from HyukjinKwon/to_timestamp_arg.
-
wangzhenhua authored
[SPARK-20718][SQL] FileSourceScanExec with different filter orders should be the same after canonicalization ## What changes were proposed in this pull request? Since `constraints` in `QueryPlan` is a set, the order of filters can differ. Usually this is ok because of canonicalization. However, in `FileSourceScanExec`, its data filters and partition filters are sequences, and their orders are not canonicalized. So `def sameResult` returns different results for different orders of data/partition filters. This leads to, e.g. different decision for `ReuseExchange`, and thus results in unstable performance. ## How was this patch tested? Added a new test for `FileSourceScanExec.sameResult`. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17959 from wzhfy/canonicalizeFileSourceScanExec.
-
- May 11, 2017
-
-
liuxian authored
## What changes were proposed in this pull request? spark-sql>select bround(12.3, 2); spark-sql>NULL For this case, the expected result is 12.3, but it is null. So ,when the second parameter is bigger than "decimal.scala", the result is not we expected. "round" function has the same problem. This PR can solve the problem for both of them. ## How was this patch tested? unit test cases in MathExpressionsSuite and MathFunctionsSuite Author: liuxian <liu.xian3@zte.com.cn> Closes #17906 from 10110346/wip_lx_0509.
-
Liang-Chi Hsieh authored
[SPARK-20399][SQL] Add a config to fallback string literal parsing consistent with old sql parser behavior ## What changes were proposed in this pull request? The new SQL parser is introduced into Spark 2.0. All string literals are unescaped in parser. Seems it bring an issue regarding the regex pattern string. The following codes can reproduce it: val data = Seq("\u0020\u0021\u0023", "abc") val df = data.toDF() // 1st usage: works in 1.6 // Let parser parse pattern string val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'") // 2nd usage: works in 1.6, 2.x // Call Column.rlike so the pattern string is a literal which doesn't go through parser val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$")) // In 2.x, we need add backslashes to make regex pattern parsed correctly val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'") Follow the discussion in #17736, this patch adds a config to fallback to 1.6 string literal parsing and mitigate migration issue. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17887 from viirya/add-config-fallback-string-parsing.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr supported a DDL-formatted string in `DataFrameReader.schema`. This fix could make users easily define a schema without importing `o.a.spark.sql.types._`. ## How was this patch tested? Added tests in `DataFrameReaderWriterSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17719 from maropu/SPARK-20431.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr added `withName` in `UserDefinedFunction` for printing UDF names in EXPLAIN ## How was this patch tested? Added tests in `UDFSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17712 from maropu/SPARK-20416.
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr added parsing rules to support aliases in table value functions. The previous pr (#17666) has been reverted because of the regression. This new pr fixed the regression and add tests in `SQLQueryTestSuite`. ## How was this patch tested? Added tests in `PlanParserSuite` and `SQLQueryTestSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17928 from maropu/SPARK-20311-3.
-
Wenchen Fan authored
## What changes were proposed in this pull request? `RuntimeReplaceable` always has a constructor with the expression to replace with, and this constructor should not be the function builder. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17876 from cloud-fan/minor.
-
Robert Kruszewski authored
## What changes were proposed in this pull request? Don't convert toRdd when doing toJSON ## How was this patch tested? Existing unit tests Author: Robert Kruszewski <robertk@palantir.com> Closes #14615 from robert3005/robertk/correct-tojson.
-
- May 10, 2017
-
-
Ala Luszczak authored
## What changes were proposed in this pull request? https://github.com/apache/spark/commit/b486ffc86d8ad6c303321dcf8514afee723f61f8 left behind references to "number of generated rows" metrics, that should have been removed. ## How was this patch tested? Existing unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #17939 from ala/SPARK-19447-fix.
-
wangzhenhua authored
## What changes were proposed in this pull request? In filter estimation, we update column stats for those columns in filter condition. However, if the number of rows decreases after the filter (i.e. the overall selectivity is less than 1), we need to update (scale down) the number of distinct values (NDV) for all columns, no matter they are in filter conditions or not. This pr also fixes the inconsistency of rounding mode for ndv and rowCount. ## How was this patch tested? Added new tests. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17918 from wzhfy/scaleDownNdvAfterFilter.
-