aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BYXin Wu2016-09-145-12/+398
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values. However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC). This PR is to support this new feature. ## How was this patch tested? New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Xin Wu <xinwu@us.ibm.com> Closes #14842 from xwu0226/SPARK-10747.
* [MINOR][SQL] Add missing functions for some options in SQLConf and use them ↵hyukjinkwon2016-09-1512-31/+49
| | | | | | | | | | | | | | | | | | | | where applicable ## What changes were proposed in this pull request? I first thought they are missing because they are kind of hidden options but it seems they are just missing. For example, `spark.sql.parquet.mergeSchema` is documented in [sql-programming-guide.md](https://github.com/apache/spark/blob/master/docs/sql-programming-guide.md) but this function is missing whereas many options such as `spark.sql.join.preferSortMergeJoin` are not documented but have its own function individually. So, this PR suggests making them consistent by adding the missing functions for some options in `SQLConf` and use them where applicable, in order to make them more readable. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14678 from HyukjinKwon/sqlconf-cleanup.
* [SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same ↵Josh Rosen2016-09-142-14/+7
| | | | | | | | | | | | | | | | | | | | in Python ## What changes were proposed in this pull request? In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing. The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd.<some-pyspark-conversions>.toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.). In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations. ## How was this patch tested? Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries. Author: Josh Rosen <joshrosen@databricks.com> Closes #15068 from JoshRosen/pyspark-collect-limit.
* [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Oncegatorsmile2016-09-149-33/+38
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? As explained in https://github.com/apache/spark/pull/14797: >Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. We should not optimize the query in CTAS more than once. For example, ```Scala spark.range(99, 101).createOrReplaceTempView("tab1") val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") checkAnswer(spark.table("tab2"), sql(sqlStmt)) ``` Before this PR, the results do not match ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![100,100.000000000000000000] [100,null] [99,99.000000000000000000] [99,99.000000000000000000] ``` After this PR, the results match. ``` +---+----------------------+ |id |num | +---+----------------------+ |99 |99.000000000000000000 | |100|100.000000000000000000| +---+----------------------+ ``` In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`. ### How was this patch tested? Added a test Author: gatorsmile <gatorsmile@gmail.com> Closes #15048 from gatorsmile/ctasOptimized.
* [SPARK-17445][DOCS] Reference an ASF page as the main place to find ↵Sean Owen2016-09-143-12/+10
| | | | | | | | | | | | | | | | | | third-party packages ## What changes were proposed in this pull request? Point references to spark-packages.org to https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects This will be accompanied by a parallel change to the spark-website repo, and additional changes to this wiki. ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #15075 from srowen/SPARK-17445.
* [SPARK-17480][SQL] Improve performance by removing or caching List.length ↵Ergin Seyfe2016-09-142-8/+5
| | | | | | | | | | | | | | | | | | | | which is O(n) ## What changes were proposed in this pull request? Scala's List.length method is O(N) and it makes the gatherCompressibilityStats function O(N^2). Eliminate the List.length calls by writing it in Scala way. https://github.com/scala/scala/blob/2.10.x/src/library/scala/collection/LinearSeqOptimized.scala#L36 As suggested. Extended the fix to HiveInspectors and AggregationIterator classes as well. ## How was this patch tested? Profiled a Spark job and found that CompressibleColumnBuilder is using 39% of the CPU. Out of this 39% CompressibleColumnBuilder->gatherCompressibilityStats is using 23% of it. 6.24% of the CPU is spend on List.length which is called inside gatherCompressibilityStats. After this change we started to save 6.24% of the CPU. Author: Ergin Seyfe <eseyfe@fb.com> Closes #15032 from seyfe/gatherCompressibilityStats.
* [SPARK-17530][SQL] Add Statistics into DESCRIBE FORMATTEDgatorsmile2016-09-141-0/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? Statistics is missing in the output of `DESCRIBE FORMATTED`. This PR is to add it. After the PR, the output will be like: ``` +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ |key |string |null | |value |string |null | | | | | |# Detailed Table Information| | | |Database: |default | | |Owner: |xiaoli | | |Create Time: |Tue Sep 13 14:36:57 PDT 2016 | | |Last Access Time: |Wed Dec 31 16:00:00 PST 1969 | | |Location: |file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-9982e1db-df17-4376-a140-dbbee0203d83/texttable| | |Table Type: |MANAGED | | |Statistics: |sizeInBytes=5812, rowCount=500, isBroadcastable=false | | |Table Parameters: | | | | rawDataSize |-1 | | | numFiles |1 | | | transient_lastDdlTime |1473802620 | | | totalSize |5812 | | | COLUMN_STATS_ACCURATE |false | | | numRows |-1 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat | | |Compressed: |No | | |Storage Desc Parameters: | | | | serialization.format |1 | | +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ ``` Also improve the output of statistics in `DESCRIBE EXTENDED` by removing duplicate `Statistics`. Below is the example after the PR: ``` +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ |key |string |null | |value |string |null | | | | | |# Detailed Table Information|CatalogTable( Table: `default`.`texttable` Owner: xiaoli Created: Tue Sep 13 14:38:43 PDT 2016 Last Access: Wed Dec 31 16:00:00 PST 1969 Type: MANAGED Schema: [StructField(key,StringType,true), StructField(value,StringType,true)] Provider: hive Properties: [rawDataSize=-1, numFiles=1, transient_lastDdlTime=1473802726, totalSize=5812, COLUMN_STATS_ACCURATE=false, numRows=-1] Statistics: sizeInBytes=5812, rowCount=500, isBroadcastable=false Storage(Location: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-8ea5c5a0-5680-4778-91cb-c6334cf8a708/texttable, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]))| | +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ ``` ### How was this patch tested? Manually tested. Author: gatorsmile <gatorsmile@gmail.com> Closes #15083 from gatorsmile/descFormattedStats.
* [SPARK-17515] CollectLimit.execute() should perform per-partition limitsJosh Rosen2016-09-132-1/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? CollectLimit.execute() incorrectly omits per-partition limits, leading to performance regressions in case this case is hit (which should not happen in normal operation, but can occur in some cases (see #15068 for one example). ## How was this patch tested? Regression test in SQLQuerySuite that asserts the number of records scanned from the input RDD. Author: Josh Rosen <joshrosen@databricks.com> Closes #15070 from JoshRosen/SPARK-17515.
* [SPARK-17474] [SQL] fix python udf in TakeOrderedAndProjectExecDavies Liu2016-09-123-12/+12
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When there is any Python UDF in the Project between Sort and Limit, it will be collected into TakeOrderedAndProjectExec, ExtractPythonUDFs failed to pull the Python UDFs out because QueryPlan.expressions does not include the expression inside Option[Seq[Expression]]. Ideally, we should fix the `QueryPlan.expressions`, but tried with no luck (it always run into infinite loop). In PR, I changed the TakeOrderedAndProjectExec to no use Option[Seq[Expression]] to workaround it. cc JoshRosen ## How was this patch tested? Added regression test. Author: Davies Liu <davies@databricks.com> Closes #15030 from davies/all_expr.
* [SPARK-17415][SQL] Better error message for driver-side broadcast join OOMsSameer Agarwal2016-09-111-31/+42
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a trivial patch that catches all `OutOfMemoryError` while building the broadcast hash relation and rethrows it by wrapping it in a nice error message. ## How was this patch tested? Existing Tests Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14979 from sameeragarwal/broadcast-join-error.
* [SPARK-17330][SPARK UT] Clean up spark-warehouse in UTtone-zhang2016-09-111-0/+2
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Check the database warehouse used in Spark UT, and remove the existing database file before run the UT (SPARK-8368). ## How was this patch tested? Run Spark UT with the command for several times: ./build/sbt -Pyarn -Phadoop-2.6 -Phive -Phive-thriftserver "test-only *HiveSparkSubmitSuit*" Without the patch, the test case can be passed only at the first time, and always failed from the second time. With the patch the test case always can be passed correctly. Author: tone-zhang <tone.zhang@linaro.org> Closes #14894 from tone-zhang/issue1.
* [SPARK-15453][SQL] FileSourceScanExec to extract `outputOrdering` informationTejas Patil2016-09-101-17/+62
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-15453 Extracting sort ordering information in `FileSourceScanExec` so that planner can make use of it. My motivation to make this change was to get Sort Merge join in par with Hive's Sort-Merge-Bucket join when the source tables are bucketed + sorted. Query: ``` val df = (0 until 16).map(i => (i % 8, i * 2, i.toString)).toDF("i", "j", "k").coalesce(1) df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table8") df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table9") context.sql("SELECT * FROM table8 a JOIN table9 b ON a.j=b.j AND a.k=b.k").explain(true) ``` Before: ``` == Physical Plan == *SortMergeJoin [j#120, k#121], [j#123, k#124], Inner :- *Sort [j#120 ASC, k#121 ASC], false, 0 : +- *Project [i#119, j#120, k#121] : +- *Filter (isnotnull(k#121) && isnotnull(j#120)) : +- *FileScan orc default.table8[i#119,j#120,k#121] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string> +- *Sort [j#123 ASC, k#124 ASC], false, 0 +- *Project [i#122, j#123, k#124] +- *Filter (isnotnull(k#124) && isnotnull(j#123)) +- *FileScan orc default.table9[i#122,j#123,k#124] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string> ``` After: (note that the `Sort` step is no longer there) ``` == Physical Plan == *SortMergeJoin [j#49, k#50], [j#52, k#53], Inner :- *Project [i#48, j#49, k#50] : +- *Filter (isnotnull(k#50) && isnotnull(j#49)) : +- *FileScan orc default.table8[i#48,j#49,k#50] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string> +- *Project [i#51, j#52, k#53] +- *Filter (isnotnull(j#52) && isnotnull(k#53)) +- *FileScan orc default.table9[i#51,j#52,k#53] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(j), IsNotNull(k)], ReadSchema: struct<i:int,j:int,k:string> ``` ## How was this patch tested? Added a test case in `JoinSuite`. Ran all other tests in `JoinSuite` Author: Tejas Patil <tejasp@fb.com> Closes #14864 from tejasapatil/SPARK-15453_smb_optimization.
* [SPARK-17354] [SQL] Partitioning by dates/timestamps should work with ↵hyukjinkwon2016-09-093-3/+57
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Parquet vectorized reader ## What changes were proposed in this pull request? This PR fixes `ColumnVectorUtils.populate` so that Parquet vectorized reader can read partitioned table with dates/timestamps. This works fine with Parquet normal reader. This is being only called within [VectorizedParquetRecordReader.java#L185](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java#L185). When partition column types are explicitly given to `DateType` or `TimestampType` (rather than inferring the type of partition column), this fails with the exception below: ``` 16/09/01 10:30:07 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 6) java.lang.ClassCastException: java.lang.Integer cannot be cast to java.sql.Date at org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate(ColumnVectorUtils.java:89) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:185) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:204) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReader$1.apply(ParquetFileFormat.scala:362) ... ``` ## How was this patch tested? Unit tests in `SQLQuerySuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14919 from HyukjinKwon/SPARK-17354.
* [SPARK-17432][SQL] PreprocessDDL should respect case sensitivity when ↵Wenchen Fan2016-09-082-1/+13
| | | | | | | | | | | | | | | | | | checking duplicated columns ## What changes were proposed in this pull request? In `PreprocessDDL` we will check if table columns are duplicated. However, this checking ignores case sensitivity config(it's always case-sensitive) and lead to different result between `HiveExternalCatalog` and `InMemoryCatalog`. `HiveExternalCatalog` will throw exception because hive metastore is always case-nonsensitive, and `InMemoryCatalog` is fine. This PR fixes it. ## How was this patch tested? a new test in DDLSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #14994 from cloud-fan/check-dup.
* [SPARK-17427][SQL] function SIZE should return -1 when parameter is nullDaoyuan Wang2016-09-071-6/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `select size(null)` returns -1 in Hive. In order to be compatible, we should return `-1`. ## How was this patch tested? unit test in `CollectionFunctionsSuite` and `DataFrameFunctionsSuite`. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #14991 from adrian-wang/size.
* [SPARK-17359][SQL][MLLIB] Use ArrayBuffer.+=(A) instead of ↵Liwei Lin2016-09-073-6/+6
| | | | | | | | | | | | | | | | ArrayBuffer.append(A) in performance critical paths ## What changes were proposed in this pull request? We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing. ## How was this patch tested? N/A Author: Liwei Lin <lwlin7@gmail.com> Closes #14914 from lw-lin/append_to_plus_eq_v2.
* [SPARK-17372][SQL][STREAMING] Avoid serialization issues by using Arrays to ↵Tathagata Das2016-09-066-18/+65
| | | | | | | | | | | | | | | | | | | | | | | | | | save file names in FileStreamSource ## What changes were proposed in this pull request? When we create a filestream on a directory that has partitioned subdirs (i.e. dir/x=y/), then ListingFileCatalog.allFiles returns the files in the dir as Seq[String] which internally is a Stream[String]. This is because of this [line](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L93), where a LinkedHashSet.values.toSeq returns Stream. Then when the [FileStreamSource](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L79) filters this Stream[String] to remove the seen files, it creates a new Stream[String], which has a filter function that has a $outer reference to the FileStreamSource (in Scala 2.10). Trying to serialize this Stream[String] causes NotSerializableException. This will happened even if there is just one file in the dir. Its important to note that this behavior is different in Scala 2.11. There is no $outer reference to FileStreamSource, so it does not throw NotSerializableException. However, with a large sequence of files (tested with 10000 files), it throws StackOverflowError. This is because how Stream class is implemented. Its basically like a linked list, and attempting to serialize a long Stream requires *recursively* going through linked list, thus resulting in StackOverflowError. In short, across both Scala 2.10 and 2.11, serialization fails when both the following conditions are true. - file stream defined on a partitioned directory - directory has 10k+ files The right solution is to convert the seq to an array before writing to the log. This PR implements this fix in two ways. - Changing all uses for HDFSMetadataLog to ensure Array is used instead of Seq - Added a `require` in HDFSMetadataLog such that it is never used with type Seq ## How was this patch tested? Added unit test that test that ensures the file stream source can handle with 10000 files. This tests fails in both Scala 2.10 and 2.11 with different failures as indicated above. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14987 from tdas/SPARK-17372.
* [SPARK-16922] [SPARK-17211] [SQL] make the address of values portable in ↵Davies Liu2016-09-062-8/+75
| | | | | | | | | | | | | | | | | | LongToUnsafeRowMap ## What changes were proposed in this pull request? In LongToUnsafeRowMap, we use offset of a value as pointer, stored in a array also in the page for chained values. The offset is not portable, because Platform.LONG_ARRAY_OFFSET will be different with different JVM Heap size, then the deserialized LongToUnsafeRowMap will be corrupt. This PR will change to use portable address (without Platform.LONG_ARRAY_OFFSET). ## How was this patch tested? Added a test case with random generated keys, to improve the coverage. But this test is not a regression test, that could require a Spark cluster that have at least 32G heap in driver or executor. Author: Davies Liu <davies@databricks.com> Closes #14927 from davies/longmap.
* [SPARK-17374][SQL] Better error messages when parsing JSON using DataFrameReaderSean Zhong2016-09-062-2/+66
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds better error messages for malformed record when reading a JSON file using DataFrameReader. For example, for query: ``` import org.apache.spark.sql.types._ val corruptRecords = spark.sparkContext.parallelize("""{"a":{, b:3}""" :: Nil) val schema = StructType(StructField("a", StringType, true) :: Nil) val jsonDF = spark.read.schema(schema).json(corruptRecords) ``` **Before change:** We silently replace corrupted line with null ``` scala> jsonDF.show +----+ | a| +----+ |null| +----+ ``` **After change:** Add an explicit warning message: ``` scala> jsonDF.show 16/09/02 14:43:16 WARN JacksonParser: Found at least one malformed records (sample: {"a":{, b:3}). The JSON reader will replace all malformed records with placeholder null in current PERMISSIVE parser mode. To find out which corrupted records have been replaced with null, please use the default inferred schema instead of providing a custom schema. Code example to print all malformed records (scala): =================================================== // The corrupted record exists in column _corrupt_record. val parsedJson = spark.read.json("/path/to/json/file/test.json") +----+ | a| +----+ |null| +----+ ``` ### ## How was this patch tested? Unit test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14929 from clockfly/logwarning_if_schema_not_contain_corrupted_record.
* [SPARK-17356][SQL] Fix out of memory issue when generating JSON for TreeNodeSean Zhong2016-09-061-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? class `org.apache.spark.sql.types.Metadata` is widely used in mllib to store some ml attributes. `Metadata` is commonly stored in `Alias` expression. ``` case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, override val isGenerated: java.lang.Boolean = false) ``` The `Metadata` can take a big memory footprint since the number of attributes is big ( in scale of million). When `toJSON` is called on `Alias` expression, the `Metadata` will also be converted to a big JSON string. If a plan contains many such kind of `Alias` expressions, it may trigger out of memory error when `toJSON` is called, since converting all `Metadata` references to JSON will take huge memory. With this PR, we will skip scanning Metadata when doing JSON conversion. For a reproducer of the OOM, and analysis, please look at jira https://issues.apache.org/jira/browse/SPARK-17356. ## How was this patch tested? Existing tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14915 from clockfly/json_oom.
* [SPARK-17361][SQL] file-based external table without path should not be createdWenchen Fan2016-09-067-41/+45
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation. This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`. Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups. ## How was this patch tested? existing tests and new test in `CatalogSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14921 from cloud-fan/check-path.
* [SPARK-17358][SQL] Cached table(parquet/orc) should be shard between beelinesYadong Qi2016-09-063-7/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Cached table(parquet/orc) couldn't be shard between beelines, because the `sameResult` method used by `CacheManager` always return false(`sparkSession` are different) when compare two `HadoopFsRelation` in different beelines. So we make `sparkSession` a curry parameter. ## How was this patch tested? Beeline1 ``` 1: jdbc:hive2://localhost:10000> CACHE TABLE src_pqt; +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (5.143 seconds) 1: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#49, value#50] +- InMemoryRelation [key#49, value#50], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string> | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Beeline2 ``` 0: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt; +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | plan | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ | == Physical Plan == InMemoryTableScan [key#68, value#69] +- InMemoryRelation [key#68, value#69], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt` +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string> | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+ ``` Author: Yadong Qi <qiyadong2010@gmail.com> Closes #14913 from watermen/SPARK-17358.
* [SPARK-17072][SQL] support table-level statistics generation and storing ↵wangzhenhua2016-09-059-43/+94
| | | | | | | | | | | | | | | | | | | | | | | into/loading from metastore ## What changes were proposed in this pull request? 1. Support generation table-level statistics for - hive tables in HiveExternalCatalog - data source tables in HiveExternalCatalog - data source tables in InMemoryCatalog. 2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side. 3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl. 4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats). ## How was this patch tested? add unit tests Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wangzhenhua@huawei.com> Closes #14712 from wzhfy/tableStats.
* [SPARK-17394][SQL] should not allow specify database in table/view name ↵Wenchen Fan2016-09-054-39/+22
| | | | | | | | | | | | | | | | | | | after RENAME TO ## What changes were proposed in this pull request? It's really weird that we allow users to specify database in both from table name and to table name in `ALTER TABLE RENAME TO`, while logically we can't support rename a table to a different database. Both postgres and MySQL disallow this syntax, it's reasonable to follow them and simply our code. ## How was this patch tested? new test in `DDLCommandSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14955 from cloud-fan/rename.
* [SPARK-17393][SQL] Error Handling when CTAS Against the Same Data Source ↵gatorsmile2016-09-051-26/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Table Using Overwrite Mode ### What changes were proposed in this pull request? When we trying to read a table and then write to the same table using the `Overwrite` save mode, we got a very confusing error message: For example, ```Scala Seq((1, 2)).toDF("i", "j").write.saveAsTable("tab1") table("tab1").write.mode(SaveMode.Overwrite).saveAsTable("tab1") ``` ``` Job aborted. org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply$mcV$sp ... Caused by: org.apache.spark.SparkException: Task failed while writing rows at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:266) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143) at org.apache.spark.sql.execution.datasources ``` After the PR, we will issue an `AnalysisException`: ``` Cannot overwrite table `tab1` that is also being read from ``` ### How was this patch tested? Added test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14954 from gatorsmile/ctasQueryAnalyze.
* [SPARK-17324][SQL] Remove Direct Usage of HiveClient in InsertIntoHiveTablegatorsmile2016-09-041-2/+1
| | | | | | | | | | | | ### What changes were proposed in this pull request? This is another step to get rid of HiveClient from `HiveSessionState`. All the metastore interactions should be through `ExternalCatalog` interface. However, the existing implementation of `InsertIntoHiveTable ` still requires Hive clients. This PR is to remove HiveClient by moving the metastore interactions into `ExternalCatalog`. ### How was this patch tested? Existing test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14888 from gatorsmile/removeClientFromInsertIntoHiveTable.
* [SPARK-17335][SQL] Fix ArrayType and MapType CatalogString.Herman van Hovell2016-09-031-75/+99
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? the `catalogString` for `ArrayType` and `MapType` currently calls the `simpleString` method on its children. This is a problem when the child is a struct, the `struct.simpleString` implementation truncates the number of fields it shows (25 at max). This breaks the generation of a proper `catalogString`, and has shown to cause errors while writing to Hive. This PR fixes this by providing proper `catalogString` implementations for `ArrayData` or `MapData`. ## How was this patch tested? Added testing for `catalogString` to `DataTypeSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14938 from hvanhovell/SPARK-17335.
* [SPARK-17298][SQL] Require explicit CROSS join for cartesian productsSrinath Shankar2016-09-0320-62/+288
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Require the use of CROSS join syntax in SQL (and a new crossJoin DataFrame API) to specify explicit cartesian products between relations. By cartesian product we mean a join between relations R and S where there is no join condition involving columns from both R and S. If a cartesian product is detected in the absence of an explicit CROSS join, an error must be thrown. Turning on the "spark.sql.crossJoin.enabled" configuration flag will disable this check and allow cartesian products without an explicit CROSS join. The new crossJoin DataFrame API must be used to specify explicit cross joins. The existing join(DataFrame) method will produce a INNER join that will require a subsequent join condition. That is df1.join(df2) is equivalent to select * from df1, df2. ## How was this patch tested? Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used. Author: Srinath Shankar <srinath@databricks.com> Closes #14866 from srinathshankar/crossjoin.
* [SPARK-16334] Reusing same dictionary column for decoding consecutive row ↵Sameer Agarwal2016-09-021-16/+38
| | | | | | | | | | | | | | | | groups shouldn't throw an error ## What changes were proposed in this pull request? This patch fixes a bug in the vectorized parquet reader that's caused by re-using the same dictionary column vector while reading consecutive row groups. Specifically, this issue manifests for a certain distribution of dictionary/plain encoded data while we read/populate the underlying bit packed dictionary data into a column-vector based data structure. ## How was this patch tested? Manually tested on datasets provided by the community. Thanks to Chris Perluss and Keith Kraus for their invaluable help in tracking down this issue! Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14941 from sameeragarwal/parquet-exception-2.
* [SPARK-17230] [SQL] Should not pass optimized query into QueryExecution in ↵Davies Liu2016-09-026-5/+24
| | | | | | | | | | | | | | | | | | | | | | | | DataFrameWriter ## What changes were proposed in this pull request? Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. Ideally, we should make all the analyzer rules all idempotent, that may require lots of effort to double checking them one by one (may be not easy). An easier approach could be never feed a optimized plan into Analyzer, this PR fix the case for RunnableComand, they will be optimized, during execution, the passed `query` will also be passed into QueryExecution again. This PR make these `query` not part of the children, so they will not be optimized and analyzed again. Right now, we did not know a logical plan is optimized or not, we could introduce a flag for that, and make sure a optimized logical plan will not be analyzed again. ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #14797 from davies/fix_writer.
* [SPARK-17351] Refactor JDBCRDD to expose ResultSet -> Seq[Row] utility methodsJosh Rosen2016-09-023-309/+335
| | | | | | | | | | | | | | | | | | | | | | | | This patch refactors the internals of the JDBC data source in order to allow some of its code to be re-used in an automated comparison testing harness. Here are the key changes: - Move the JDBC `ResultSetMetadata` to `StructType` conversion logic from `JDBCRDD.resolveTable()` to the `JdbcUtils` object (as a new `getSchema(ResultSet, JdbcDialect)` method), allowing it to be applied on `ResultSet`s that are created elsewhere. - Move the `ResultSet` to `InternalRow` conversion methods from `JDBCRDD` to `JdbcUtils`: - It makes sense to move the `JDBCValueGetter` type and `makeGetter` functions here given that their write-path counterparts (`JDBCValueSetter`) are already in `JdbcUtils`. - Add an internal `resultSetToSparkInternalRows` method which takes a `ResultSet` and schema and returns an `Iterator[InternalRow]`. This effectively extracts the main loop of `JDBCRDD` into its own method. - Add a public `resultSetToRows` method to `JdbcUtils`, which wraps the minimal machinery around `resultSetToSparkInternalRows` in order to allow it to be called from outside of a Spark job. - Make `JdbcDialect.get` into a `DeveloperApi` (`JdbcDialect` itself is already a `DeveloperApi`). Put together, these changes enable the following testing pattern: ```scala val jdbResultSet: ResultSet = conn.prepareStatement(query).executeQuery() val resultSchema: StructType = JdbcUtils.getSchema(jdbResultSet, JdbcDialects.get("jdbc:postgresql")) val jdbcRows: Seq[Row] = JdbcUtils.resultSetToRows(jdbResultSet, schema).toSeq checkAnswer(sparkResult, jdbcRows) // in a test case ``` Author: Josh Rosen <joshrosen@databricks.com> Closes #14907 from JoshRosen/modularize-jdbc-internals.
* [SPARK-16984][SQL] don't try whole dataset immediately when first partition ↵Robert Kruszewski2016-09-022-15/+23
| | | | | | | | | | | | | | | | doesn't have… ## What changes were proposed in this pull request? Try increase number of partitions to try so we don't revert to all. ## How was this patch tested? Empirically. This is common case optimization. Author: Robert Kruszewski <robertk@palantir.com> Closes #14573 from robert3005/robertk/execute-take-backoff.
* [SQL][DOC][MINOR] Add (Scala-specific) and (Java-specific)Jacek Laskowski2016-09-021-0/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adds (Scala-specific) and (Java-specific) to Scaladoc. ## How was this patch tested? local build Author: Jacek Laskowski <jacek@japila.pl> Closes #14891 from jaceklaskowski/scala-specifics.
* [SPARK-16302][SQL] Set the right number of partitions for reading data from ↵Lianhui Wang2016-09-011-1/+4
| | | | | | | | | | | | | a local collection. follow #13137 This pr sets the right number of partitions when reading data from a local collection. Query 'val df = Seq((1, 2)).toDF("key", "value").count' always use defaultParallelism tasks. So it causes run many empty or small tasks. Manually tested and checked. Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #13979 from lianhuiwang/localTable-Parallel.
* [SPARK-16525] [SQL] Enable Row Based HashMap in HashAggregateExecQifan Pu2016-09-016-112/+319
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is the second step for the following feature: For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields). In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs. In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`. ## How was this patch tested? Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite` Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series. Author: Qifan Pu <qifan.pu@gmail.com> Author: ooq <qifan.pu@gmail.com> Closes #14176 from ooq/rowbasedfastaggmap-pr2.
* [SPARK-17355] Workaround for HIVE-14684 / HiveResultSetMetaData.isSigned ↵Josh Rosen2016-09-011-1/+10
| | | | | | | | | | | | | | | | | | | | | | | exception ## What changes were proposed in this pull request? Attempting to use Spark SQL's JDBC data source against the Hive ThriftServer results in a `java.sql.SQLException: Method` not supported exception from `org.apache.hive.jdbc.HiveResultSetMetaData.isSigned`. Here are two user reports of this issue: - https://stackoverflow.com/questions/34067686/spark-1-5-1-not-working-with-hive-jdbc-1-2-0 - https://stackoverflow.com/questions/32195946/method-not-supported-in-spark I have filed [HIVE-14684](https://issues.apache.org/jira/browse/HIVE-14684) to attempt to fix this in Hive by implementing the isSigned method, but in the meantime / for compatibility with older JDBC drivers I think we should add special-case error handling to work around this bug. This patch updates `JDBCRDD`'s `ResultSetMetadata` to schema conversion to catch the "Method not supported" exception from Hive and return `isSigned = true`. I believe that this is safe because, as far as I know, Hive does not support unsigned numeric types. ## How was this patch tested? Tested manually against a Spark Thrift Server. Author: Josh Rosen <joshrosen@databricks.com> Closes #14911 from JoshRosen/hive-jdbc-workaround.
* [SPARK-16461][SQL] Support partition batch pruning with `<=>` predicate in ↵hyukjinkwon2016-09-012-0/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | InMemoryTableScanExec ## What changes were proposed in this pull request? It seems `EqualNullSafe` filter was missed for batch pruneing partitions in cached tables. It seems supporting this improves the performance roughly 5 times faster. Running the codes below: ```scala test("Null-safe equal comparison") { val N = 20000000 val df = spark.range(N).repartition(20) val benchmark = new Benchmark("Null-safe equal comparison", N) df.createOrReplaceTempView("t") spark.catalog.cacheTable("t") sql("select id from t where id <=> 1").collect() benchmark.addCase("Null-safe equal comparison", 10) { _ => sql("select id from t where id <=> 1").collect() } benchmark.run() } ``` produces the results below: **Before:** ``` Running benchmark: Null-safe equal comparison Running case: Null-safe equal comparison Stopped after 10 iterations, 2098 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5 Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Null-safe equal comparison 204 / 210 98.1 10.2 1.0X ``` **After:** ``` Running benchmark: Null-safe equal comparison Running case: Null-safe equal comparison Stopped after 10 iterations, 478 ms Java HotSpot(TM) 64-Bit Server VM 1.8.0_45-b14 on Mac OS X 10.11.5 Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz Null-safe equal comparison: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ Null-safe equal comparison 42 / 48 474.1 2.1 1.0X ``` ## How was this patch tested? Unit tests in `PartitionBatchPruningSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14117 from HyukjinKwon/SPARK-16461.
* [SPARK-17331][CORE][MLLIB] Avoid allocating 0-length arraysSean Owen2016-09-011-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid allocating some 0-length arrays, esp. in UTF8String, and by using Array.empty in Scala over Array[T]() ## How was this patch tested? Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #14895 from srowen/SPARK-17331.
* [SPARK-17263][SQL] Add hexadecimal literal parsingHerman van Hovell2016-09-012-5/+44
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`. If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`. Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data. ## How was this patch tested? Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14832 from hvanhovell/SPARK-17263.
* [SPARK-17257][SQL] the physical plan of CREATE TABLE or CTAS should take ↵Wenchen Fan2016-09-014-101/+75
| | | | | | | | | | | | | | | | CatalogTable ## What changes were proposed in this pull request? This is kind of a follow-up of https://github.com/apache/spark/pull/14482 . As we put `CatalogTable` in the logical plan directly, it makes sense to let physical plans take `CatalogTable` directly, instead of extracting some fields of `CatalogTable` in planner and then construct a new `CatalogTable` in physical plan. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14823 from cloud-fan/create-table.
* [SPARK-17353][SPARK-16943][SPARK-16942][SQL] Fix multiple bugs in CREATE ↵gatorsmile2016-09-011-12/+45
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | TABLE LIKE command ### What changes were proposed in this pull request? The existing `CREATE TABLE LIKE` command has multiple issues: - The generated table is non-empty when the source table is a data source table. The major reason is the data source table is using the table property `path` to store the location of table contents. Currently, we keep it unchanged. Thus, we still create the same table with the same location. - The table type of the generated table is `EXTERNAL` when the source table is an external Hive Serde table. Currently, we explicitly set it to `MANAGED`, but Hive is checking the table property `EXTERNAL` to decide whether the table is `EXTERNAL` or not. (See https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1407-L1408) Thus, the created table is still `EXTERNAL`. - When the source table is a `VIEW`, the metadata of the generated table contains the original view text and view original text. So far, this does not break anything, but it could cause something wrong in Hive. (For example, https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1405-L1406) - The issue regarding the table `comment`. To follow what Hive does, the table comment should be cleaned, but the column comments should be still kept. - The `INDEX` table is not supported. Thus, we should throw an exception in this case. - `owner` should not be retained. `ToHiveTable` set it [here](https://github.com/apache/spark/blob/e679bc3c1cd418ef0025d2ecbc547c9660cac433/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala#L793) no matter which value we set in `CatalogTable`. We set it to an empty string for avoiding the confusing output in Explain. - Add a support for temp tables - Like Hive, we should not copy the table properties from the source table to the created table, especially for the statistics-related properties, which could be wrong in the created table. - `unsupportedFeatures` should not be copied from the source table. The created table does not have these unsupported features. - When the type of source table is a view, the target table is using the default format of data source tables: `spark.sql.sources.default`. This PR is to fix the above issues. ### How was this patch tested? Improve the test coverage by adding more test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14531 from gatorsmile/createTableLike.
* [SPARK-16283][SQL] Implements percentile_approx aggregation function which ↵Sean Zhong2016-09-011-0/+226
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | supports partial aggregation. ## What changes were proposed in this pull request? This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`. ### Syntax: ``` # Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory. percentile_approx(col, percentage [, accuracy]) # Returns percentile value array at given percentage value array percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) ``` ### Features: 1. This function supports partial aggregation. 2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint. 3. This function supports window function aggregation. ### Example usages: ``` ## Returns the 25th percentile value, with default accuracy SELECT percentile_approx(col, 0.25) FROM table ## Returns an array of percentile value (25th, 50th, 75th), with default accuracy SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table ## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error SELECT percentile_approx(col, 0.25, 100) FROM table ## Returns the 25th, and 50th percentile values, with custom accuracy value 100 SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table ``` ### NOTE: 1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)` 2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal. ## How was this patch tested? Unit test, and Sql query test. ## Acknowledgement 1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue. Author: Sean Zhong <seanzhong@databricks.com> Closes #14868 from clockfly/appro_percentile_try_2.
* revert PR#10896 and PR#14865Wenchen Fan2016-09-018-277/+223
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? according to the discussion in the original PR #10896 and the new approach PR #14876 , we decided to revert these 2 PRs and go with the new approach. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14909 from cloud-fan/revert.
* [SPARK-17180][SPARK-17309][SPARK-17323][SQL] create AlterViewAsCommand to ↵Wenchen Fan2016-08-312-57/+83
| | | | | | | | | | | | | | | | | | | | | | handle ALTER VIEW AS ## What changes were proposed in this pull request? Currently we use `CreateViewCommand` to implement ALTER VIEW AS, which has 3 bugs: 1. SPARK-17180: ALTER VIEW AS should alter temp view if view name has no database part and temp view exists 2. SPARK-17309: ALTER VIEW AS should issue exception if view does not exist. 3. SPARK-17323: ALTER VIEW AS should keep the previous table properties, comment, create_time, etc. The root cause is, ALTER VIEW AS is quite different from CREATE VIEW, we need different code path to handle them. However, in `CreateViewCommand`, there is no way to distinguish ALTER VIEW AS and CREATE VIEW, we have to introduce extra flag. But instead of doing this, I think a more natural way is to separate the ALTER VIEW AS logic into a new command. ## How was this patch tested? new tests in SQLViewSuite Author: Wenchen Fan <wenchen@databricks.com> Closes #14874 from cloud-fan/minor4.
* [MINOR][MLLIB][SQL] Clean up unused variables and unused importXin Ren2016-08-304-4/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Clean up unused variables and unused import statements, unnecessary `return` and `toArray`, and some more style improvement, when I walk through the code examples. ## How was this patch tested? Testet manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14836 from keypointt/codeWalkThroughML.
* [SPARK-17264][SQL] DataStreamWriter should document that it only supports ↵Sean Owen2016-08-301-1/+1
| | | | | | | | | | | | | | | | Parquet for now ## What changes were proposed in this pull request? Clarify that only parquet files are supported by DataStreamWriter now ## How was this patch tested? (Doc build -- no functional changes to test) Author: Sean Owen <sowen@cloudera.com> Closes #14860 from srowen/SPARK-17264.
* [SPARK-17289][SQL] Fix a bug to satisfy sort requirements in partial ↵Takeshi YAMAMURO2016-08-302-2/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | aggregations ## What changes were proposed in this pull request? Partial aggregations are generated in `EnsureRequirements`, but the planner fails to check if partial aggregation satisfies sort requirements. For the following query: ``` val df2 = (0 to 1000).map(x => (x % 2, x.toString)).toDF("a", "b").createOrReplaceTempView("t2") spark.sql("select max(b) from t2 group by a").explain(true) ``` Now, the SortAggregator won't insert Sort operator before partial aggregation, this will break sort-based partial aggregation. ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- LocalTableScan [a#5, b#6] ``` Actually, a correct plan is: ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- *Sort [a#5 ASC], false, 0 +- LocalTableScan [a#5, b#6] ``` ## How was this patch tested? Added tests in `PlannerSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14865 from maropu/SPARK-17289.
* [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastoreDavies Liu2016-08-293-29/+150
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions). It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default). ## How was this patch tested? Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster). Author: Davies Liu <davies@databricks.com> Closes #14607 from davies/repair_batch.
* [SPARK-17271][SQL] Planner adds un-necessary Sort even if child ordering is ↵Tejas Patil2016-08-282-2/+49
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | semantically same as required ordering ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-17271 Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253 `SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects. eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")` Expression in required SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId, qualifier = Some("a") ) ``` Expression in child SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId) ``` Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order. This PR includes following changes: - Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals) - Fixed `EnsureRequirements` to use semantic comparison of SortOrder ## How was this patch tested? - Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite` Author: Tejas Patil <tejasp@fb.com> Closes #14841 from tejasapatil/SPARK-17271_sort_order_equals_bug.
* [SPARK-15382][SQL] Fix a bug in sampling with replacementTakeshi YAMAMURO2016-08-272-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr to fix a bug below in sampling with replacement ``` val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b") df.sample(true, 2.0).withColumn("c", monotonically_increasing_id).select($"c").show +---+ | c| +---+ | 0| | 1| | 1| | 1| | 2| +---+ ``` ## How was this patch tested? Added a test in `DataFrameSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14800 from maropu/FixSampleBug.