Hi,

Since the csv source currently supports ascii-compatible charset, so I
guess shift-jis also works well.
You could check Hyukjin's comment in
https://issues.apache.org/jira/browse/SPARK-21289 for more info.


On Wed, Aug 16, 2017 at 2:54 PM, Han-Cheol Cho <prian...@gmail.com> wrote:

> My apologies,
>
> It was a problem of our Hadoop cluster.
> When we tested the same code on another cluster (HDP-based), it worked
> without any problem.
>
> ```scala
> ## make sjis text
> cat a.txt
> 8月データだけでやってみよう
> nkf -W -s a.txt >b.txt
> cat b.txt
> 87n%G!<%?$@$1$G$d$C$F$_$h$&
> nkf -s -w b.txt
> 8月データだけでやってみよう
> hdfs dfs -put a.txt b.txt
>
> ## YARN mode test
> spark.read.option("encoding", "utf-8").csv("a.txt").show(1)
> +--------------+
> |           _c0|
> +--------------+
> |8月データだけでやってみよう|
> +--------------+
>
> spark.read.option("encoding", "sjis").csv("b.txt").show(1)
> +--------------+
> |           _c0|
> +--------------+
> |8月データだけでやってみよう|
> +--------------+
>
> spark.read.option("encoding", "utf-8").option("multiLine",
> true).csv("a.txt").show(1)
> +--------------+
> |           _c0|
> +--------------+
> |8月データだけでやってみよう|
> +--------------+
>
> spark.read.option("encoding", "sjis").option("multiLine",
> true).csv("b.txt").show(1)
> +--------------+
> |           _c0|
> +--------------+
> |8月データだけでやってみよう|
> +--------------+
> ```
>
> I am still digging the root cause and will share it later :-)
>
> Best wishes,
> Han-Choel
>
>
> On Wed, Aug 16, 2017 at 1:32 PM, Han-Cheol Cho <prian...@gmail.com> wrote:
>
>> Dear Spark ML members,
>>
>>
>> I experienced a trouble in using "multiLine" option to load CSV data with
>> Shift-JIS encoding.
>> When option("multiLine", true) is specified, option("encoding",
>> "encoding-name") just doesn't work anymore.
>>
>>
>> In CSVDataSource.scala file, I found that MultiLineCSVDataSource.readFile()
>> method doesn't use parser.options.charset at all.
>>
>> object MultiLineCSVDataSource extends CSVDataSource {
>>   override val isSplitable: Boolean = false
>>
>>   override def readFile(
>>       conf: Configuration,
>>       file: PartitionedFile,
>>       parser: UnivocityParser,
>>       schema: StructType): Iterator[InternalRow] = {
>>     UnivocityParser.parseStream(
>>       CodecStreams.createInputStreamWithCloseResource(conf,
>> file.filePath),
>>       parser.options.headerFlag,
>>       parser,
>>       schema)
>>   }
>>   ...
>>
>> On the other hand, TextInputCSVDataSource.readFile() method uses it:
>>
>>   override def readFile(
>>       conf: Configuration,
>>       file: PartitionedFile,
>>       parser: UnivocityParser,
>>       schema: StructType): Iterator[InternalRow] = {
>>     val lines = {
>>       val linesReader = new HadoopFileLinesReader(file, conf)
>>       Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ =>
>> linesReader.close()))
>>       linesReader.map { line =>
>>         new String(line.getBytes, 0, line.getLength,
>> parser.options.charset)    // <---- charset option is used here.
>>       }
>>     }
>>
>>     val shouldDropHeader = parser.options.headerFlag && file.start == 0
>>     UnivocityParser.parseIterator(lines, shouldDropHeader, parser,
>> schema)
>>   }
>>
>>
>> It seems like a bug.
>> Is there anyone who had the same problem before?
>>
>>
>> Best wishes,
>> Han-Cheol
>>
>> --
>> ==================================
>> Han-Cheol Cho, Ph.D.
>> Data scientist, Data Science Team, Data Laboratory
>> NHN Techorus Corp.
>>
>> Homepage: https://sites.google.com/site/priancho/
>> ==================================
>>
>
>
>
> --
> ==================================
> Han-Cheol Cho, Ph.D.
> Data scientist, Data Science Team, Data Laboratory
> NHN Techorus Corp.
>
> Homepage: https://sites.google.com/site/priancho/
> ==================================
>



-- 
---
Takeshi Yamamuro

Reply via email to