[ 
https://issues.apache.org/jira/browse/SPARK-26972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16783834#comment-16783834
 ] 

Jean Georges Perrin commented on SPARK-26972:
---------------------------------------------

[~srowen], [~hyukjin.kwon] - thanks guy for dealing with a rookie! 

I'll do my best to give a try against master, however:
 # the non-case sensitivity becoming case sensitivity, is that scheduled for 
v3.0 or already in v2.4.x?
 # I double checked the output, when you specify the schema: 

in 2.1.3, it crashes:
{code:java}
2019-03-04 17:17:41.854 -ERROR --- [rker for task 0] 
Logging$class.logError(Logging.scala:91): Exception in task 0.0 in stage 0.0 
(TID 0)
java.lang.NumberFormatException: For input string: "An independent study by 
Jean Georges Perrin, IIUG Board Member*"
at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
at java.lang.Integer.parseInt(Integer.java:580)
at java.lang.Integer.parseInt(Integer.java:615)
at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
at 
org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:252)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:125)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:94)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:167)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:166)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:109)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:377)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
at org.apache.spark.scheduler.Task.run(Task.scala:100)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:325)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
2019-03-04 17:17:41.876 -ERROR --- [result-getter-0] 
Logging$class.logError(Logging.scala:70): Task 0 in stage 0.0 failed 1 times; 
aborting job
Exception in thread "main" org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost 
task 0.0 in stage 0.0 (TID 0, localhost, executor driver): 
java.lang.NumberFormatException: For input string: "An independent study by 
Jean Georges Perrin, IIUG Board Member*"
at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
at java.lang.Integer.parseInt(Integer.java:580)
at java.lang.Integer.parseInt(Integer.java:615)
at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
at 
org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:252)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:125)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:94)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:167)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:166)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:109)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:377)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
at org.apache.spark.scheduler.Task.run(Task.scala:100)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:325)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)

Driver stacktrace:
at 
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1455)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1443)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1442)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at 
org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802)
at scala.Option.foreach(Option.scala:257)
at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:802)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1670)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1625)
at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1614)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:628)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1928)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1941)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:1954)
at org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:333)
at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38)
at 
org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2390)
at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)
at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2792)
at 
org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2389)
at 
org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2396)
at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2132)
at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:2131)
at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2822)
at org.apache.spark.sql.Dataset.head(Dataset.scala:2131)
at org.apache.spark.sql.Dataset.take(Dataset.scala:2346)
at org.apache.spark.sql.Dataset.showString(Dataset.scala:248)
at org.apache.spark.sql.Dataset.show(Dataset.scala:666)
at 
net.jgp.books.spark.ch07.lab110_csv_ingestion_with_schema.ComplexCsvToDataframeWithSchemaApp.start(ComplexCsvToDataframeWithSchemaApp.java:81)
at 
net.jgp.books.spark.ch07.lab110_csv_ingestion_with_schema.ComplexCsvToDataframeWithSchemaApp.main(ComplexCsvToDataframeWithSchemaApp.java:27)
Caused by: java.lang.NumberFormatException: For input string: "An independent 
study by Jean Georges Perrin, IIUG Board Member*"
at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
at java.lang.Integer.parseInt(Integer.java:580)
at java.lang.Integer.parseInt(Integer.java:615)
at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
at 
org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:252)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:125)
at 
org.apache.spark.sql.execution.datasources.csv.CSVRelation$$anonfun$csvParser$3.apply(CSVRelation.scala:94)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:167)
at 
org.apache.spark.sql.execution.datasources.csv.CSVFileFormat$$anonfun$buildReader$1$$anonfun$apply$2.apply(CSVFileFormat.scala:166)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:109)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:377)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
at org.apache.spark.scheduler.Task.run(Task.scala:100)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:325)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
{code}
In 2.2.3 & 2.4.0:
{code:java}
+---+---------+-------------------------+-----------+-----------------------+
| id|authordId|                bookTitle|releaseDate|                    url|
+---+---------+-------------------------+-----------+-----------------------+
|  1|        1|Fantastic Beasts and W...| 2016-11-18|http://amzn.to/2kup94P
|
|  2|        1|Harry Potter and the S...| 2015-10-06|http://amzn.to/2l2lSwP
|
|  3|        1|The Tales of Beedle th...| 2008-12-04|http://amzn.to/2kYezqr
|
|  4|        1|Harry Potter and the C...| 2016-10-04|http://amzn.to/2kYhL5n
|
|  5|        2|Informix 12.10 on Mac ...| 2017-04-23|http://amzn.to/2i3mthT
|
+---+---------+-------------------------+-----------+-----------------------+
only showing top 5 rows

root
 |-- id: integer (nullable = true)
 |-- authordId: integer (nullable = true)
 |-- bookTitle: string (nullable = true)
 |-- releaseDate: date (nullable = true)
 |-- url: string (nullable = true)
{code}
At least, it does not have the carriage return in the fieldname (as it comes 
from the schema).

I must admit, I like 2.0.2's behavior better, where the carriage return/new 
line is swallowed.:
{code:java}
+---+---------+----------------------------------------------------------------------------------------------------------+-----------+----------------------+
|id |authordId|bookTitle                                                        
                                         |releaseDate|url                   |
+---+---------+----------------------------------------------------------------------------------------------------------+-----------+----------------------+
|1  |1        |Fantastic Beasts and Where to Find Them: The Original Screenplay 
                                         |2016-11-18 |http://amzn.to/2kup94P|
|2  |1        |Harry Potter and the Sorcerer's Stone: The Illustrated Edition 
(Harry Potter; Book 1)                     |2015-10-06 |http://amzn.to/2l2lSwP|
|3  |1        |The Tales of Beedle the Bard, Standard Edition (Harry Potter)    
                                         |2008-12-04 |http://amzn.to/2kYezqr|
|4  |1        |Harry Potter and the Chamber of Secrets: The Illustrated Edition 
(Harry Potter; Book 2)                   |2016-10-04 |http://amzn.to/2kYhL5n|
|5  |2        |Informix 12.10 on Mac 10.12 with a dash of Java 8: The Tale of 
the Apple; the Coffee; and a Great Database|2017-04-23 |http://amzn.to/2i3mthT|
+---+---------+----------------------------------------------------------------------------------------------------------+-----------+----------------------+
only showing top 5 rows

root
 |-- id: integer (nullable = true)
 |-- authordId: integer (nullable = true)
 |-- bookTitle: string (nullable = true)
 |-- releaseDate: date (nullable = true)
 |-- url: string (nullable = true)
{code}
 

 

> Issue with CSV import and inferSchema set to true
> -------------------------------------------------
>
>                 Key: SPARK-26972
>                 URL: https://issues.apache.org/jira/browse/SPARK-26972
>             Project: Spark
>          Issue Type: Bug
>          Components: Input/Output
>    Affects Versions: 2.1.3, 2.3.3, 2.4.0
>         Environment: Java 8/Scala 2.11/MacOs
>            Reporter: Jean Georges Perrin
>            Priority: Major
>         Attachments: ComplexCsvToDataframeApp.java, 
> ComplexCsvToDataframeWithSchemaApp.java, books.csv, issue.txt, pom.xml
>
>
>  
> I found a few discrepencies while working with inferSchema set to true in CSV 
> ingestion.
> Given the following CSV in the attached books.csv:
> {noformat}
> id;authorId;title;releaseDate;link
> 1;1;Fantastic Beasts and Where to Find Them: The Original 
> Screenplay;11/18/16;http://amzn.to/2kup94P
> 2;1;*Harry Potter and the Sorcerer's Stone: The Illustrated Edition (Harry 
> Potter; Book 1)*;10/6/15;http://amzn.to/2l2lSwP
> 3;1;*The Tales of Beedle the Bard, Standard Edition (Harry 
> Potter)*;12/4/08;http://amzn.to/2kYezqr
> 4;1;*Harry Potter and the Chamber of Secrets: The Illustrated Edition (Harry 
> Potter; Book 2)*;10/4/16;http://amzn.to/2kYhL5n
> 5;2;*Informix 12.10 on Mac 10.12 with a dash of Java 8: The Tale of the 
> Apple; the Coffee; and a Great Database*;4/23/17;http://amzn.to/2i3mthT
> 6;2;*Development Tools in 2006: any Room for a 4GL-style Language?
> An independent study by Jean Georges Perrin, IIUG Board 
> Member*;12/28/16;http://amzn.to/2vBxOe1
> 7;3;Adventures of Huckleberry Finn;5/26/94;http://amzn.to/2wOeOav
> 8;3;A Connecticut Yankee in King Arthur's Court;6/17/17;http://amzn.to/2x1NuoD
> 10;4;Jacques le Fataliste;3/1/00;http://amzn.to/2uZj2KA
> 11;4;Diderot Encyclopedia: The Complete Illustrations 
> 1762-1777;;http://amzn.to/2i2zo3I
> 12;;A Woman in Berlin;7/11/06;http://amzn.to/2i472WZ
> 13;6;Spring Boot in Action;1/3/16;http://amzn.to/2hCPktW
> 14;6;Spring in Action: Covers Spring 4;11/28/14;http://amzn.to/2yJLyCk
> 15;7;Soft Skills: The software developer's life 
> manual;12/29/14;http://amzn.to/2zNnSyn
> 16;8;Of Mice and Men;;http://amzn.to/2zJjXoc
> 17;9;*Java 8 in Action: Lambdas; Streams; and functional-style 
> programming*;8/28/14;http://amzn.to/2isdqoL
> 18;12;Hamlet;6/8/12;http://amzn.to/2yRbewY
> 19;13;Pensées;12/31/1670;http://amzn.to/2jweHOG
> 20;14;*Fables choisies; mises en vers par M. de La 
> Fontaine*;9/1/1999;http://amzn.to/2yRH10W
> 21;15;Discourse on Method and Meditations on First 
> Philosophy;6/15/1999;http://amzn.to/2hwB8zc
> 22;12;Twelfth Night;7/1/4;http://amzn.to/2zPYnwo
> 23;12;Macbeth;7/1/3;http://amzn.to/2zPYnwo{noformat}
> And this Java code:
> {code:java}
> Dataset<Row> df = spark.read().format("csv")
>  .option("header", "true")
>  .option("multiline", true)
>  .option("sep", ";")
>  .option("quote", "*")
>  .option("dateFormat", "M/d/y")
>  .option("inferSchema", true)
>  .load("data/books.csv");
> df.show(7);
> df.printSchema();
> {code}
> h1. In Spark v2.0.1
> Output: 
> {noformat}
> +---+--------+--------------------+-----------+--------------------+
> | id|authorId|               title|releaseDate|                link|
> +---+--------+--------------------+-----------+--------------------+
> |  1|       1|Fantastic Beasts ...|   11/18/16|http://amzn.to/2k...|
> |  2|       1|Harry Potter and ...|    10/6/15|http://amzn.to/2l...|
> |  3|       1|The Tales of Beed...|    12/4/08|http://amzn.to/2k...|
> |  4|       1|Harry Potter and ...|    10/4/16|http://amzn.to/2k...|
> |  5|       2|Informix 12.10 on...|    4/23/17|http://amzn.to/2i...|
> |  6|       2|Development Tools...|   12/28/16|http://amzn.to/2v...|
> |  7|       3|Adventures of Huc...|.   5/26/94|http://amzn.to/2w...|
> +---+--------+--------------------+-----------+--------------------+
> only showing top 7 rows
> Dataframe's schema:
> root
> |-- id: integer (nullable = true)
> |-- authorId: integer (nullable = true)
> |-- title: string (nullable = true)
> |-- releaseDate: string (nullable = true)
> |-- link: string (nullable = true)
> {noformat}
> *This is fine and the expected output*.
> h1. Using Apache Spark v2.1.3
> Excerpt of the dataframe content: 
> {noformat}
> +--------------------+--------+--------------------+-----------+--------------------+
> | id|authorId| title|releaseDate| link|
> +--------------------+--------+--------------------+-----------+--------------------+
> | 1| 1|Fantastic Beasts ...| 11/18/16|http://amzn.to/2k...|
> | 2| 1|Harry Potter and ...| 10/6/15|http://amzn.to/2l...|
> | 3| 1|The Tales of Beed...| 12/4/08|http://amzn.to/2k...|
> | 4| 1|Harry Potter and ...| 10/4/16|http://amzn.to/2k...|
> | 5| 2|Informix 12.10 on...| 4/23/17|http://amzn.to/2i...|
> | 6| 2|Development Tools...| null| null|
> |An independent st...|12/28/16|http://amzn.to/2v...| null| null|
> +--------------------+--------+--------------------+-----------+--------------------+
> only showing top 7 rows
> Dataframe's schema:
> root
> |-- id: string (nullable = true)
> |-- authorId: string (nullable = true)
> |-- title: string (nullable = true)
> |-- releaseDate: string (nullable = true)
> |-- link: string (nullable = true){noformat}
>  The *multiline* option is *not recognized*. And, of course, the schema is 
> wrong.
> h1. Using Apache Spark v2.2.3
> Excerpt of the dataframe content: 
> {noformat}
> +---+--------+--------------------+-----------+--------------------+
> | id|authorId| title|releaseDate| link
> |
> +---+--------+--------------------+-----------+--------------------+
> | 1| 1|Fantastic Beasts ...| 11/18/16|http://amzn.to/2k...|
> | 2| 1|Harry Potter and ...| 10/6/15|http://amzn.to/2l...|
> | 3| 1|The Tales of Beed...| 12/4/08|http://amzn.to/2k...|
> | 4| 1|Harry Potter and ...| 10/4/16|http://amzn.to/2k...|
> | 5| 2|Informix 12.10 on...| 4/23/17|http://amzn.to/2i...|
> | 6| 2|Development Tools...| 12/28/16|http://amzn.to/2v...|
> | 7| 3|Adventures of Huc...| 5/26/94|http://amzn.to/2w...|
> +---+--------+--------------------+-----------+--------------------+
> only showing top 7 rows
> Dataframe's schema:
> root
> |-- id: integer (nullable = true)
> |-- authorId: integer (nullable = true)
> |-- title: string (nullable = true)
> |-- releaseDate: string (nullable = true)
> |-- link
> : string (nullable = true)
> {noformat}
>  The *link* column *has a carriage return* at the end of its name. If I run 
> and use: 
> {code:java}
> df.show(7, 90);
> {code}
> I get: 
> {noformat}
> +---+--------+------------------------------------------------------------------------------------------+-----------+-----------------------+
> | id|authorId| title|releaseDate| link
> |
> +---+--------+------------------------------------------------------------------------------------------+-----------+-----------------------+
> | 1| 1| Fantastic Beasts and Where to Find Them: The Original Screenplay| 
> 11/18/16|http://amzn.to/2kup94P
> |
> | 2| 1| Harry Potter and the Sorcerer's Stone: The Illustrated Edition (Harry 
> Potter; Book 1)| 10/6/15|http://amzn.to/2l2lSwP
> |
> | 3| 1| The Tales of Beedle the Bard, Standard Edition (Harry Potter)| 
> 12/4/08|http://amzn.to/2kYezqr
> |
> | 4| 1| Harry Potter and the Chamber of Secrets: The Illustrated Edition 
> (Harry Potter; Book 2)| 10/4/16|http://amzn.to/2kYhL5n
> |
> | 5| 2|Informix 12.10 on Mac 10.12 with a dash of Java 8: The Tale of the 
> Apple; the Coffee; a...| 4/23/17|http://amzn.to/2i3mthT
> |
> | 6| 2|Development Tools in 2006: any Room for a 4GL-style Language?
> An independent study by...| 12/28/16|http://amzn.to/2vBxOe1
> |
> | 7| 3| Adventures of Huckleberry Finn| 5/26/94|http://amzn.to/2wOeOav
> |
> +---+--------+------------------------------------------------------------------------------------------+-----------+-----------------------+
> {noformat}
> The carriage *return is added to my the last cell*.
> Same behavior in v2.3.3 and v2.4.0.
> If I add the schema, like in: 
> {code:java}
>     // Creates the schema
>     StructType schema = DataTypes.createStructType(new StructField[] {
>         DataTypes.createStructField(
>             "id",
>             DataTypes.IntegerType,
>             false),
>         DataTypes.createStructField(
>             "authordId",
>             DataTypes.IntegerType,
>             true),
>         DataTypes.createStructField(
>             "bookTitle",
>             DataTypes.StringType,
>             false),
>         DataTypes.createStructField(
>             "releaseDate",
>             DataTypes.DateType,
>             true), // nullable, but this will be ignore
>         DataTypes.createStructField(
>             "url",
>             DataTypes.StringType,
>             false) });
>     // Reads a CSV file with header, called books.csv, stores it in a 
> dataframe
>     Dataset<Row> df = spark.read().format("csv")
>         .option("header", "true")
>         .option("multiline", true)
>         .option("sep", ";")
>         .option("dateFormat", "M/d/y")
>         .option("quote", "*")
>         .schema(schema)
>         .load("data/books.csv");
> {code}
> The output is matching what is expected in any version *except version 2.1.3, 
> where Spark simply crashes*.
> All the code can be downloaded from GitHub at: 
> [https://github.com/jgperrin/net.jgp.books.sparkWithJava.ch07.]
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to