[jira] [Commented] (ORC-1482) RecordReaderImpl.evaluatePredicateProto assumes floating point stats are always present
[ https://issues.apache.org/jira/browse/ORC-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17757476#comment-17757476 ] Jason Darrell Lowe commented on ORC-1482: - An ID sounds great, but do we really need an ID before fixing this? I don't see the need to check for a writer ID for this fix, and not all files will have that new writer ID. > RecordReaderImpl.evaluatePredicateProto assumes floating point stats are > always present > --- > > Key: ORC-1482 > URL: https://issues.apache.org/jira/browse/ORC-1482 > Project: ORC > Issue Type: Bug >Affects Versions: 1.7.4 >Reporter: Jason Darrell Lowe >Priority: Major > Attachments: > part-0-45af54ac-9306-447f-8d4f-cc2f1b1cf61f-c000.snappy.orc > > > ORC-629 added custom handling of predicate pushdown on doubles, but the code > that was added blindly assumes that double statistics were present in the > file which may not have been the case. Here's the snippet of code in > question: > {code:java} > } else if (category == TypeDescription.Category.DOUBLE || > category == TypeDescription.Category.FLOAT) { > DoubleColumnStatistics dstas = (DoubleColumnStatistics) cs; > {code} > > Elsewhere in the code, there's a type check on the result of statistics > deserialization before casting, but here the type checks are missing. It > appears this should be checking for DoubleColumnStatistics before assuming > the cast will succeed, and if the expected statistics type is not present > then the predicate should not be evaluated on that column. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Updated] (ORC-1482) RecordReaderImpl.evaluatePredicateProto assumes floating point stats are always present
[ https://issues.apache.org/jira/browse/ORC-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Darrell Lowe updated ORC-1482: Attachment: part-0-45af54ac-9306-447f-8d4f-cc2f1b1cf61f-c000.snappy.orc > RecordReaderImpl.evaluatePredicateProto assumes floating point stats are > always present > --- > > Key: ORC-1482 > URL: https://issues.apache.org/jira/browse/ORC-1482 > Project: ORC > Issue Type: Bug >Affects Versions: 1.7.4 >Reporter: Jason Darrell Lowe >Priority: Major > Attachments: > part-0-45af54ac-9306-447f-8d4f-cc2f1b1cf61f-c000.snappy.orc > > > ORC-629 added custom handling of predicate pushdown on doubles, but the code > that was added blindly assumes that double statistics were present in the > file which may not have been the case. Here's the snippet of code in > question: > {code:java} > } else if (category == TypeDescription.Category.DOUBLE || > category == TypeDescription.Category.FLOAT) { > DoubleColumnStatistics dstas = (DoubleColumnStatistics) cs; > {code} > > Elsewhere in the code, there's a type check on the result of statistics > deserialization before casting, but here the type checks are missing. It > appears this should be checking for DoubleColumnStatistics before assuming > the cast will succeed, and if the expected statistics type is not present > then the predicate should not be evaluated on that column. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (ORC-1482) RecordReaderImpl.evaluatePredicateProto assumes floating point stats are always present
[ https://issues.apache.org/jira/browse/ORC-1482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17755565#comment-17755565 ] Jason Darrell Lowe commented on ORC-1482: - > For the official orc writer, if it's a DOUBLE / FLOAT type, I think it must > write DoubleColumnStatistics Yes, the Java ORC writer always writes the statistics. This error was found while trying to read an ORC file generated by another writer, RAPIDS cudf. See the discussion at [https://github.com/rapidsai/cudf/issues/13793.] While the official writer always generates stats, it's interesting that everywhere else in the reader code I looked, it is tolerant of missing statistics. The reader was tolerant of this before ORC-629. Seems like an easy thing to fix, independent of whether the spec is updated to clarify when statistics are required. > RecordReaderImpl.evaluatePredicateProto assumes floating point stats are > always present > --- > > Key: ORC-1482 > URL: https://issues.apache.org/jira/browse/ORC-1482 > Project: ORC > Issue Type: Bug >Affects Versions: 1.7.4 >Reporter: Jason Darrell Lowe >Priority: Major > > ORC-629 added custom handling of predicate pushdown on doubles, but the code > that was added blindly assumes that double statistics were present in the > file which may not have been the case. Here's the snippet of code in > question: > {code:java} > } else if (category == TypeDescription.Category.DOUBLE || > category == TypeDescription.Category.FLOAT) { > DoubleColumnStatistics dstas = (DoubleColumnStatistics) cs; > {code} > > Elsewhere in the code, there's a type check on the result of statistics > deserialization before casting, but here the type checks are missing. It > appears this should be checking for DoubleColumnStatistics before assuming > the cast will succeed, and if the expected statistics type is not present > then the predicate should not be evaluated on that column. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (ORC-1482) RecordReaderImpl.evaluatePredicateProto assumes floating point stats are always present
Jason Darrell Lowe created ORC-1482: --- Summary: RecordReaderImpl.evaluatePredicateProto assumes floating point stats are always present Key: ORC-1482 URL: https://issues.apache.org/jira/browse/ORC-1482 Project: ORC Issue Type: Bug Affects Versions: 1.7.4 Reporter: Jason Darrell Lowe ORC-629 added custom handling of predicate pushdown on doubles, but the code that was added blindly assumes that double statistics were present in the file which may not have been the case. Here's the snippet of code in question: {code:java} } else if (category == TypeDescription.Category.DOUBLE || category == TypeDescription.Category.FLOAT) { DoubleColumnStatistics dstas = (DoubleColumnStatistics) cs; {code} Elsewhere in the code, there's a type check on the result of statistics deserialization before casting, but here the type checks are missing. It appears this should be checking for DoubleColumnStatistics before assuming the cast will succeed, and if the expected statistics type is not present then the predicate should not be evaluated on that column. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (SPARK-39981) CheckOverflowInTableInsert returns exception rather than throwing it
Jason Darrell Lowe created SPARK-39981: -- Summary: CheckOverflowInTableInsert returns exception rather than throwing it Key: SPARK-39981 URL: https://issues.apache.org/jira/browse/SPARK-39981 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.4.0, 3.3.1 Reporter: Jason Darrell Lowe CheckOverflowInTableInsert added by [SPARK-39865] returns the result of QueryExecutionErrors.castingCauseOverflowErrorInTableInsert rather than throwing it. This results in a ClassCastException at runtime, since it cannot interpret the exception object as the expected value type. For example: {noformat} scala> sql("create table tiny(i tinyint)") 22/08/04 10:27:32 WARN ResolveSessionCatalog: A Hive serde table will be created as there is no table provider specified. You can set spark.sql.legacy.createHiveTableByDefault to false so that native data source table will be created instead. res0: org.apache.spark.sql.DataFrame = [] scala> sql("insert into tiny values (1000);") java.lang.ClassCastException: org.apache.spark.SparkArithmeticException cannot be cast to java.lang.Byte at scala.runtime.BoxesRunTime.unboxToByte(BoxesRunTime.java:95) at org.apache.spark.sql.catalyst.InternalRow$.$anonfun$getWriter$2(InternalRow.scala:171) at org.apache.spark.sql.catalyst.InternalRow$.$anonfun$getWriter$2$adapted(InternalRow.scala:171) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.$anonfun$fieldWriters$2(InterpretedMutableProjection.scala:76) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.$anonfun$fieldWriters$2$adapted(InterpretedMutableProjection.scala:76) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.apply(InterpretedMutableProjection.scala:103) at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$46.$anonfun$applyOrElse$75(Optimizer.scala:1994) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at scala.collection.TraversableLike.map(TraversableLike.scala:286) at scala.collection.TraversableLike.map$(TraversableLike.scala:279) at scala.collection.AbstractTraversable.map(Traversable.scala:108) at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$46.applyOrElse(Optimizer.scala:1994) at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$46.applyOrElse(Optimizer.scala:1989) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:584) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:176) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:584) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:589) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1228) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1227) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.mapChildren(InsertIntoHiveTable.scala:73) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:589) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:550) at
[jira] [Commented] (SPARK-31474) Consistancy between dayofweek/dow in extract expression and dayofweek function
[ https://issues.apache.org/jira/browse/SPARK-31474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17088704#comment-17088704 ] Jason Darrell Lowe commented on SPARK-31474: The branch-3.0 build appears to have been broken by this commit. {noformat} [INFO] --- scala-maven-plugin:4.3.0:compile (scala-compile-first) @ spark-catalyst_2.12 --- [INFO] Using incremental compilation using Mixed compile order [INFO] Compiler bridge file: /home/user/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.12-1.3.1-bin_2.12.10__52.0-1.3.1_20191012T045515.jar [INFO] Compiling 304 Scala sources and 98 Java sources to /home/user/src/spark/sql/catalyst/target/scala-2.12/classes ... [ERROR] [Error] /home/user/src/spark/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:140: not found: type Timestamp [ERROR] one error found {noformat} The patch removes the {{Timestamp}} import from datetimeExpressions.scala but there are still references to that type within the file. [~cloud_fan] was this cherry-picked to branch-3.0 without building it? > Consistancy between dayofweek/dow in extract expression and dayofweek function > -- > > Key: SPARK-31474 > URL: https://issues.apache.org/jira/browse/SPARK-31474 > Project: Spark > Issue Type: Improvement > Components: SQL >Affects Versions: 3.0.0 >Reporter: Kent Yao >Assignee: Kent Yao >Priority: Major > Fix For: 3.0.0 > > > {code:sql} > spark-sql> SELECT extract(dayofweek from '2009-07-26'); > 1 > spark-sql> SELECT extract(dow from '2009-07-26'); > 0 > spark-sql> SELECT extract(isodow from '2009-07-26'); > 7 > spark-sql> SELECT dayofweek('2009-07-26'); > 1 > spark-sql> SELECT weekday('2009-07-26'); > 6 > {code} > Currently, there are 4 types of day-of-week range: > the function dayofweek(2.3.0) and extracting dayofweek(2.4.0) result as of > Sunday(1) to Saturday(7) > extracting dow(3.0.0) results as of Sunday(0) to Saturday(6) > extracting isodow (3.0.0) results as of Monday(1) to Sunday(7) > the function weekday(2.4.0) results as of Monday(0) to Sunday(6) > Actually, extracting dayofweek and dow are both derived from PostgreSQL but > have different meanings. > https://issues.apache.org/jira/browse/SPARK-23903 > https://issues.apache.org/jira/browse/SPARK-28623 -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (MAPREDUCE-7241) FileInputFormat listStatus with less memory footprint
[ https://issues.apache.org/jira/browse/MAPREDUCE-7241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Darrell Lowe updated MAPREDUCE-7241: -- Fix Version/s: 3.4.0 Hadoop Flags: Reviewed Resolution: Fixed Status: Resolved (was: Patch Available) Thanks for the contribution, [~dengzh]! I committed this to trunk. > FileInputFormat listStatus with less memory footprint > - > > Key: MAPREDUCE-7241 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-7241 > Project: Hadoop Map/Reduce > Issue Type: Improvement > Components: job submission >Affects Versions: 2.6.1 >Reporter: Zhihua Deng >Assignee: Zhihua Deng >Priority: Major > Fix For: 3.4.0 > > Attachments: MAPREDUCE-7241.03.patch, MAPREDUCE-7241.04.patch, > MAPREDUCE-7241.05.patch, MAPREDUCE-7241.trunk.02.patch, > MAPREDUCE-7241.trunk.patch, filestatus.png > > > This case sometimes sees in hive when user issues queries over all partitions > by mistakes. The file status cached when listing status could accumulate to > over 3g. After digging into the dumped memory, the LocatedBlock occupies > about 50%(sometimes over 60%) memory that retained by LocatedFileStatus, as > shows followed, > !filestatus.png! > Right now we only extract the block locations info from LocatedFileStatus, > the datanode infos(types) or block token are not taken into account. So there > is no need to cache LocatedBlock, as do like this: > BlockLocation[] blockLocations = dedup(stat.getBlockLocations()); > LocatedFileStatus shrink = new LocatedFileStatus(stat, blockLocations); > private static BlockLocation[] dup(BlockLocation[] blockLocations) { > BlockLocation[] copyLocs = new BlockLocation[blockLocations.length]; > int i = 0; > for (BlockLocation location : blockLocations) > { copyLocs[i++] = new BlockLocation(location); } > return copyLocs; > } > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org
[jira] [Assigned] (MAPREDUCE-7241) FileInputFormat listStatus with less memory footprint
[ https://issues.apache.org/jira/browse/MAPREDUCE-7241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Darrell Lowe reassigned MAPREDUCE-7241: - Assignee: Zhihua Deng > FileInputFormat listStatus with less memory footprint > - > > Key: MAPREDUCE-7241 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-7241 > Project: Hadoop Map/Reduce > Issue Type: Improvement > Components: job submission >Affects Versions: 2.6.1 >Reporter: Zhihua Deng >Assignee: Zhihua Deng >Priority: Major > Attachments: MAPREDUCE-7241.03.patch, MAPREDUCE-7241.04.patch, > MAPREDUCE-7241.05.patch, MAPREDUCE-7241.trunk.02.patch, > MAPREDUCE-7241.trunk.patch, filestatus.png > > > This case sometimes sees in hive when user issues queries over all partitions > by mistakes. The file status cached when listing status could accumulate to > over 3g. After digging into the dumped memory, the LocatedBlock occupies > about 50%(sometimes over 60%) memory that retained by LocatedFileStatus, as > shows followed, > !filestatus.png! > Right now we only extract the block locations info from LocatedFileStatus, > the datanode infos(types) or block token are not taken into account. So there > is no need to cache LocatedBlock, as do like this: > BlockLocation[] blockLocations = dedup(stat.getBlockLocations()); > LocatedFileStatus shrink = new LocatedFileStatus(stat, blockLocations); > private static BlockLocation[] dup(BlockLocation[] blockLocations) { > BlockLocation[] copyLocs = new BlockLocation[blockLocations.length]; > int i = 0; > for (BlockLocation location : blockLocations) > { copyLocs[i++] = new BlockLocation(location); } > return copyLocs; > } > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org
[jira] [Commented] (YARN-3238) Connection timeouts to nodemanagers are retried at multiple levels
[ https://issues.apache.org/jira/browse/YARN-3238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17032359#comment-17032359 ] Jason Darrell Lowe commented on YARN-3238: -- This did go into 3.0.0-alpha1 (see commit 92d67ace3248930c0c0335070cc71a480c566a36) but was later superceded by YARN-4414. > Connection timeouts to nodemanagers are retried at multiple levels > -- > > Key: YARN-3238 > URL: https://issues.apache.org/jira/browse/YARN-3238 > Project: Hadoop YARN > Issue Type: Bug >Affects Versions: 2.6.0 >Reporter: Jason Darrell Lowe >Assignee: Jason Darrell Lowe >Priority: Blocker > Labels: 2.6.1-candidate > Fix For: 2.7.0, 2.6.1, 3.0.0-alpha1 > > Attachments: YARN-3238.001.patch > > > The IPC layer will retry connection timeouts automatically (see Client.java), > but we are also retrying them with YARN's RetryPolicy put in place when the > NM proxy is created. This causes a two-level retry mechanism where the IPC > layer has already retried quite a few times (45 by default) for each YARN > RetryPolicy error that is retried. The end result is that NM clients can > wait a very, very long time for the connection to finally fail. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org
[jira] [Commented] (SPARK-30530) CSV load followed by "is null" filter produces incorrect results
[ https://issues.apache.org/jira/browse/SPARK-30530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17017088#comment-17017088 ] Jason Darrell Lowe commented on SPARK-30530: The regressed behavior was introduced by this commit: {noformat} commit 4e50f0291f032b4a5c0b46ed01fdef14e4cbb050 Author: Maxim Gekk Date: Thu Jan 16 13:10:08 2020 +0900 [SPARK-30323][SQL] Support filters pushdown in CSV datasource {noformat} [~maxgekk] would you take a look? > CSV load followed by "is null" filter produces incorrect results > > > Key: SPARK-30530 > URL: https://issues.apache.org/jira/browse/SPARK-30530 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: Jason Darrell Lowe >Priority: Major > > Trying to filter on is null from values loaded from a CSV file has regressed > recently and now produces incorrect results. > Given a CSV file with the contents: > {noformat:title=floats.csv} > 100.0,1.0, > 200.0,, > 300.0,3.0, > 1.0,4.0, > ,4.0, > 500.0,, > ,6.0, > -500.0,50.5 > {noformat} > Filtering this data for the first column being null should return exactly two > rows, but it is returning extraneous rows with nulls: > {noformat} > scala> val schema = StructType(Array(StructField("floats", FloatType, > true),StructField("more_floats", FloatType, true))) > schema: org.apache.spark.sql.types.StructType = > StructType(StructField(floats,FloatType,true), > StructField(more_floats,FloatType,true)) > scala> val df = spark.read.schema(schema).csv("floats.csv") > df: org.apache.spark.sql.DataFrame = [floats: float, more_floats: float] > scala> df.filter("floats is null").show > +--+---+ > |floats|more_floats| > +--+---+ > | null| null| > | null| null| > | null| null| > | null| null| > | null|4.0| > | null| null| > | null|6.0| > +--+---+ > {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-30530) CSV load followed by "is null" filter produces incorrect results
Jason Darrell Lowe created SPARK-30530: -- Summary: CSV load followed by "is null" filter produces incorrect results Key: SPARK-30530 URL: https://issues.apache.org/jira/browse/SPARK-30530 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.0.0 Reporter: Jason Darrell Lowe Trying to filter on is null from values loaded from a CSV file has regressed recently and now produces incorrect results. Given a CSV file with the contents: {noformat:title=floats.csv} 100.0,1.0, 200.0,, 300.0,3.0, 1.0,4.0, ,4.0, 500.0,, ,6.0, -500.0,50.5 {noformat} Filtering this data for the first column being null should return exactly two rows, but it is returning extraneous rows with nulls: {noformat} scala> val schema = StructType(Array(StructField("floats", FloatType, true),StructField("more_floats", FloatType, true))) schema: org.apache.spark.sql.types.StructType = StructType(StructField(floats,FloatType,true), StructField(more_floats,FloatType,true)) scala> val df = spark.read.schema(schema).csv("floats.csv") df: org.apache.spark.sql.DataFrame = [floats: float, more_floats: float] scala> df.filter("floats is null").show +--+---+ |floats|more_floats| +--+---+ | null| null| | null| null| | null| null| | null| null| | null|4.0| | null| null| | null|6.0| +--+---+ {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30049) SQL fails to parse when comment contains an unmatched quote character
[ https://issues.apache.org/jira/browse/SPARK-30049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16998331#comment-16998331 ] Jason Darrell Lowe commented on SPARK-30049: Found some time to track this down to the following commit which first regressed the parsing behavior: {noformat} commit 148cd26799c69ab9cfdc2b3b8000a194c12518b8 (HEAD, refs/bisect/bad) Author: Yuming Wang Date: Sat Oct 12 22:21:14 2019 -0700 [SPARK-26321][SQL] Port HIVE-15297: Hive should not split semicolon within quoted string literals ## What changes were proposed in this pull request? This pr port [HIVE-15297](https://issues.apache.org/jira/browse/HIVE-15297) to fix **spark-sql** should not split semicolon within quoted string literals. ## How was this patch tested? unit tests and manual tests: ![image](https://user-images.githubusercontent.com/5399861/60395592-5666ea00-9b68-11e9-99dc-0e8ea98de32b.png) Closes #25018 from wangyum/SPARK-26321. Authored-by: Yuming Wang Signed-off-by: Yuming Wang {noformat} [~yumwang] would you mind taking a look? > SQL fails to parse when comment contains an unmatched quote character > - > > Key: SPARK-30049 > URL: https://issues.apache.org/jira/browse/SPARK-30049 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: Jason Darrell Lowe >Priority: Major > > A SQL statement that contains a comment with an unmatched quote character can > lead to a parse error. These queries parsed correctly in older versions of > Spark. For example, here's an excerpt from an interactive spark-sql session > on a recent Spark-3.0.0-SNAPSHOT build (commit > e23c135e568d4401a5659bc1b5ae8fc8bf147693): > {noformat} > spark-sql> SELECT 1 -- someone's comment here > > ; > Error in query: > extraneous input ';' expecting (line 2, pos 0) > == SQL == > SELECT 1 -- someone's comment here > ; > ^^^ > {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30049) SQL fails to parse when comment contains an unmatched quote character
[ https://issues.apache.org/jira/browse/SPARK-30049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16986108#comment-16986108 ] Jason Darrell Lowe commented on SPARK-30049: I just tested SQLite, and it supports it: {noformat} sqlite> SELECT 1 -- someone's comment here ...> ; 1 {noformat} It's telling that the apostrophe character within the comment decides whether Spark parses it properly or not. I don't recall apostrophe or quote characters within SQL comments being significant, so this seems to be a bug in the Spark SQL parser. > SQL fails to parse when comment contains an unmatched quote character > - > > Key: SPARK-30049 > URL: https://issues.apache.org/jira/browse/SPARK-30049 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: Jason Darrell Lowe >Priority: Major > > A SQL statement that contains a comment with an unmatched quote character can > lead to a parse error. These queries parsed correctly in older versions of > Spark. For example, here's an excerpt from an interactive spark-sql session > on a recent Spark-3.0.0-SNAPSHOT build (commit > e23c135e568d4401a5659bc1b5ae8fc8bf147693): > {noformat} > spark-sql> SELECT 1 -- someone's comment here > > ; > Error in query: > extraneous input ';' expecting (line 2, pos 0) > == SQL == > SELECT 1 -- someone's comment here > ; > ^^^ > {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-30049) SQL fails to parse when comment contains an unmatched quote character
[ https://issues.apache.org/jira/browse/SPARK-30049?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16982858#comment-16982858 ] Jason Darrell Lowe commented on SPARK-30049: Interestingly, if the comment appears on its own line then the query parses correctly. For example, this query works: {noformat} SELECT 1 -- someone's comment here ; {noformat} > SQL fails to parse when comment contains an unmatched quote character > - > > Key: SPARK-30049 > URL: https://issues.apache.org/jira/browse/SPARK-30049 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 3.0.0 >Reporter: Jason Darrell Lowe >Priority: Major > > A SQL statement that contains a comment with an unmatched quote character can > lead to a parse error. These queries parsed correctly in older versions of > Spark. For example, here's an excerpt from an interactive spark-sql session > on a recent Spark-3.0.0-SNAPSHOT build (commit > e23c135e568d4401a5659bc1b5ae8fc8bf147693): > {noformat} > spark-sql> SELECT 1 -- someone's comment here > > ; > Error in query: > extraneous input ';' expecting (line 2, pos 0) > == SQL == > SELECT 1 -- someone's comment here > ; > ^^^ > {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-30049) SQL fails to parse when comment contains an unmatched quote character
Jason Darrell Lowe created SPARK-30049: -- Summary: SQL fails to parse when comment contains an unmatched quote character Key: SPARK-30049 URL: https://issues.apache.org/jira/browse/SPARK-30049 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 3.0.0 Reporter: Jason Darrell Lowe A SQL statement that contains a comment with an unmatched quote character can lead to a parse error. These queries parsed correctly in older versions of Spark. For example, here's an excerpt from an interactive spark-sql session on a recent Spark-3.0.0-SNAPSHOT build (commit e23c135e568d4401a5659bc1b5ae8fc8bf147693): {noformat} spark-sql> SELECT 1 -- someone's comment here > ; Error in query: extraneous input ';' expecting (line 2, pos 0) == SQL == SELECT 1 -- someone's comment here ; ^^^ {noformat} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org