aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17513][SQL] Make StreamExecution garbage-collect its metadatapetermaxlee2016-09-201-0/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235. This is a resubmission of 15126, which was based on work by frreiss in #15067, but fixed the test case along with some typos. ## How was this patch tested? A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request. Author: petermaxlee <petermaxlee@gmail.com> Closes #15166 from petermaxlee/SPARK-17513-2.
* [SPARK-17549][SQL] Revert "[] Only collect table size stat in driver for ↵Yin Huai2016-09-201-14/+0
| | | | | | | | | | cached relation." This reverts commit 39e2bad6a866d27c3ca594d15e574a1da3ee84cc because of the problem mentioned at https://issues.apache.org/jira/browse/SPARK-17549?focusedCommentId=15505060&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15505060 Author: Yin Huai <yhuai@databricks.com> Closes #15157 from yhuai/revert-SPARK-17549.
* [SPARK-15698][SQL][STREAMING] Add the ability to remove the old MetadataLog ↵jerryshao2016-09-202-18/+116
| | | | | | | | | | | | | | | | in FileStreamSource ## What changes were proposed in this pull request? Current `metadataLog` in `FileStreamSource` will add a checkpoint file in each batch but do not have the ability to remove/compact, which will lead to large number of small files when running for a long time. So here propose to compact the old logs into one file. This method is quite similar to `FileStreamSinkLog` but simpler. ## How was this patch tested? Unit test added. Author: jerryshao <sshao@hortonworks.com> Closes #13513 from jerryshao/SPARK-15698.
* [SPARK-17502][SQL] Fix Multiple Bugs in DDL Statements on Temporary Viewsgatorsmile2016-09-201-1/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example, ``` Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`'; ``` - When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example, ``` Attempted to unset non-existent property 'p' in table '`testView`'; ``` - When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error: ``` ANALYZE TABLE is not supported for Project ``` - When inserting into a temporary view that is generated from `Range`, we will get the following error message: ``` assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false +- Project [1 AS 1#20] +- OneRowRelation$ ``` This PR is to fix the above four issues. ### How was this patch tested? Added multiple test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #15054 from gatorsmile/tempViewDDL.
* Revert "[SPARK-17513][SQL] Make StreamExecution garbage-collect its metadata"Wenchen Fan2016-09-201-24/+0
| | | | This reverts commit be9d57fc9d8b10e4234c01c06ed43fd7dd12c07b.
* [SPARK-17513][SQL] Make StreamExecution garbage-collect its metadatapetermaxlee2016-09-191-0/+24
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235. This is based on work by frreiss in #15067, but fixed the test case along with some typos. ## How was this patch tested? A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request. Author: petermaxlee <petermaxlee@gmail.com> Author: frreiss <frreiss@us.ibm.com> Closes #15126 from petermaxlee/SPARK-17513.
* [SPARK-17571][SQL] AssertOnQuery.condition should always return Boolean valuepetermaxlee2016-09-183-4/+10
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? AssertOnQuery has two apply constructor: one that accepts a closure that returns boolean, and another that accepts a closure that returns Unit. This is actually very confusing because developers could mistakenly think that AssertOnQuery always require a boolean return type and verifies the return result, when indeed the value of the last statement is ignored in one of the constructors. This pull request makes the two constructor consistent and always require boolean value. It will overall make the test suites more robust against developer errors. As an evidence for the confusing behavior, this change also identified a bug with an existing test case due to file system time granularity. This pull request fixes that test case as well. ## How was this patch tested? This is a test only change. Author: petermaxlee <petermaxlee@gmail.com> Closes #15127 from petermaxlee/SPARK-17571.
* [SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null values properlyLiwei Lin2016-09-182-21/+35
| | | | | | | | | | | | | | | | | | | | ## Problem CSV in Spark 2.0.0: - does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6; - does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903. ## What changes were proposed in this pull request? This patch makes changes to read all empty values back as `null`s. ## How was this patch tested? New test cases. Author: Liwei Lin <lwlin7@gmail.com> Closes #14118 from lw-lin/csv-cast-null.
* [SPARK-17541][SQL] fix some DDL bugs about table management when same-name ↵Wenchen Fan2016-09-183-0/+98
| | | | | | | | | | | | | | | | | | | | | | temp view exists ## What changes were proposed in this pull request? In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are: 1. `CREATE TABLE USING` will fail if a same-name temp view exists 2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists 3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists. These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch ## How was this patch tested? new regression tests Author: Wenchen Fan <wenchen@databricks.com> Closes #15099 from cloud-fan/fix-view.
* [SPARK-17518][SQL] Block Users to Specify the Internal Data Source Provider Hivegatorsmile2016-09-181-0/+7
| | | | | | | | | | | | ### What changes were proposed in this pull request? In Spark 2.1, we introduced a new internal provider `hive` for telling Hive serde tables from data source tables. This PR is to block users to specify this in `DataFrameWriter` and SQL APIs. ### How was this patch tested? Added a test case Author: gatorsmile <gatorsmile@gmail.com> Closes #15073 from gatorsmile/formatHive.
* Correct fetchsize property name in docsDaniel Darabos2016-09-171-3/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replace `fetchSize` with `fetchsize` in the docs. ## How was this patch tested? I manually tested `fetchSize` and `fetchsize`. The latter has an effect. See also [`JdbcUtils.scala#L38`](https://github.com/apache/spark/blob/v2.0.0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L38) for the definition of the property. Author: Daniel Darabos <darabos.daniel@gmail.com> Closes #14975 from darabos/patch-3.
* [SPARK-17549][SQL] Only collect table size stat in driver for cached relation.Marcelo Vanzin2016-09-161-0/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. On a mildly related change, I'm also adding code to catch exceptions in the code generator since Janino was breaking with the test data I tried this patch on. Tested with unit tests and by doing a count a very wide table (20k columns) with many partitions. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15112 from vanzin/SPARK-17549.
* [SPARK-17426][SQL] Refactor `TreeNode.toJSON` to avoid OOM when converting ↵Sean Zhong2016-09-161-136/+0
| | | | | | | | | | | | | | | | | | | | | | | unknown fields to JSON ## What changes were proposed in this pull request? This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like: 1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error. 2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error. For a code example, please check the Jira description of SPARK-17426. In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type. ## How was this patch tested? Unit test. Author: Sean Zhong <seanzhong@databricks.com> Closes #14990 from clockfly/json_oom2.
* [SPARK-17458][SQL] Alias specified for aggregates in a pivot are not honoredAndrew Ray2016-09-151-0/+11
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change preserves aliases that are given for pivot aggregations ## How was this patch tested? New unit test Author: Andrew Ray <ray.andrew@gmail.com> Closes #15111 from aray/SPARK-17458.
* [SPARK-17429][SQL] use ImplicitCastInputTypes with function Length岑玉海2016-09-151-4/+6
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? select length(11); select length(2.0); these sql will return errors, but hive is ok. this PR will support casting input types implicitly for function length the correct result is: select length(11) return 2 select length(2.0) return 3 Author: 岑玉海 <261810726@qq.com> Author: cenyuhai <cenyuhai@didichuxing.com> Closes #15014 from cenyuhai/SPARK-17429.
* [SPARK-17114][SQL] Fix aggregates grouped by literals with empty inputHerman van Hovell2016-09-152-0/+68
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row). This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys. ## How was this patch tested? Added tests to `SQLQueryTestSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15101 from hvanhovell/SPARK-17114-3.
* [SPARK-17440][SPARK-17441] Fixed Multiple Bugs in ALTER TABLEgatorsmile2016-09-151-14/+49
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? For the following `ALTER TABLE` DDL, we should issue an exception when the target table is a `VIEW`: ```SQL ALTER TABLE viewName SET LOCATION '/path/to/your/lovely/heart' ALTER TABLE viewName SET SERDE 'whatever' ALTER TABLE viewName SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName ADD IF NOT EXISTS PARTITION (a='4', b='8') ALTER TABLE viewName DROP IF EXISTS PARTITION (a='2') ALTER TABLE viewName RECOVER PARTITIONS ALTER TABLE viewName PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p') ``` In addition, `ALTER TABLE RENAME PARTITION` is unable to handle data source tables, just like the other `ALTER PARTITION` commands. We should issue an exception instead. ### How was this patch tested? Added a few test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #15004 from gatorsmile/altertable.
* [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BYXin Wu2016-09-143-1/+339
| | | | | | | | | | | | | | | | | ## 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.
* [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Oncegatorsmile2016-09-141-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### 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-142-9/+6
| | | | | | | | | | | | | | | | | | 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-17515] CollectLimit.execute() should perform per-partition limitsJosh Rosen2016-09-131-0/+9
| | | | | | | | | | | | | | ## 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-121-2/+2
| | | | | | | | | | | | | | | | ## 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-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-17354] [SQL] Partitioning by dates/timestamps should work with ↵hyukjinkwon2016-09-091-1/+48
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-081-0/+7
| | | | | | | | | | | | | | | | | | 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-17372][SQL][STREAMING] Avoid serialization issues by using Arrays to ↵Tathagata Das2016-09-062-10/+50
| | | | | | | | | | | | | | | | | | | | | | | | | | 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-061-0/+56
| | | | | | | | | | | | | | | | | | 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-061-1/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-061-0/+17
| | | | | | | | | | | | | | | | | | ## 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-061-1/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-051-0/+26
| | | | | | | | | | | | | | | | | | | | | | | 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-052-34/+10
| | | | | | | | | | | | | | | | | | | 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-17298][SQL] Require explicit CROSS join for cartesian productsSrinath Shankar2016-09-0312-22/+252
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-17230] [SQL] Should not pass optimized query into QueryExecution in ↵Davies Liu2016-09-021-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | 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-16525] [SQL] Enable Row Based HashMap in HashAggregateExecQifan Pu2016-09-013-10/+102
| | | | | | | | | | | | | | | | | | | | | ## 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-16461][SQL] Support partition batch pruning with `<=>` predicate in ↵hyukjinkwon2016-09-011-0/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-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-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-012-71/+21
| | | | | | | | | | | | | | ## 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-17289][SQL] Fix a bug to satisfy sort requirements in partial ↵Takeshi YAMAMURO2016-08-301-1/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-291-2/+11
| | | | | | | | | | | | | | | | ## 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-281-1/+39
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-271-0/+7
| | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-17235][SQL] Support purging of old logs in MetadataLogpetermaxlee2016-08-261-4/+23
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds a purge interface to MetadataLog, and an implementation in HDFSMetadataLog. The purge function is currently unused, but I will use it to purge old execution and file source logs in follow-up patches. These changes are required in a production structured streaming job that runs for a long period of time. ## How was this patch tested? Added a unit test case in HDFSMetadataLogSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14802 from petermaxlee/SPARK-17235.
* [SPARK-17246][SQL] Add BigDecimal literalHerman van Hovell2016-08-262-1/+29
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values. ## How was this patch tested? Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14819 from hvanhovell/SPARK-17246.
* [SPARK-17165][SQL] FileStreamSource should not track the list of seen files ↵petermaxlee2016-08-262-3/+113
| | | | | | | | | | | | | | | | indefinitely ## What changes were proposed in this pull request? Before this change, FileStreamSource uses an in-memory hash set to track the list of files processed by the engine. The list can grow indefinitely, leading to OOM or overflow of the hash set. This patch introduces a new user-defined option called "maxFileAge", default to 24 hours. If a file is older than this age, FileStreamSource will purge it from the in-memory map that was used to track the list of files that have been processed. ## How was this patch tested? Added unit tests for the underlying utility, and also added an end-to-end test to validate the purge in FileStreamSourceSuite. Also verified the new test cases would fail when the timeout was set to a very large number. Author: petermaxlee <petermaxlee@gmail.com> Closes #14728 from petermaxlee/SPARK-17165.
* [SPARK-17192][SQL] Issue Exception when Users Specify the Partitioning ↵gatorsmile2016-08-261-5/+12
| | | | | | | | | | | | | | | | | | Columns without a Given Schema ### What changes were proposed in this pull request? Address the comments by yhuai in the original PR: https://github.com/apache/spark/pull/14207 First, issue an exception instead of logging a warning when users specify the partitioning columns without a given schema. Second, refactor the codes a little. ### How was this patch tested? Fixed the test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14572 from gatorsmile/followup16552.
* [SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify ↵hyukjinkwon2016-08-261-2/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | all unsupported types in CSV ## What changes were proposed in this pull request? This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV. In more details, this PR, - Enables the tests for `TimestampType` for JSON and This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below: ```scala val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime println(d.toString) ``` ``` Fri Dec 28 00:00:00 KST 899 ``` However, since we use `FastDateFormat`, it seems we are safe now. ```scala val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000") println(d) ``` ``` Tue Jan 01 00:00:00 PST 900 ``` - Verifies all unsupported types in CSV There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types. ## How was this patch tested? Tests in `JsonHadoopFsRelation` and `CSVSuite` Author: hyukjinkwon <gurwls223@gmail.com> Closes #14829 from HyukjinKwon/SPARK-16216-followup.