Skip to content

Conversation

@ScrapCodes
Copy link
Member

@ScrapCodes ScrapCodes commented Jul 12, 2016

What changes were proposed in this pull request?

In multiple text analysis problems, it is not often desirable for the rows to be split by "\n". There exists a wholeText reader for RDD API, and this JIRA just adds the same support for Dataset API.

How was this patch tested?

Added relevant new tests for both scala and Java APIs

@SparkQA
Copy link

SparkQA commented Jul 12, 2016

Test build #62158 has finished for PR 14151 at commit bd2936d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) extends Iterator[Text]
    • class WholeTextFileFormat extends TextFileFormat

@SparkQA
Copy link

SparkQA commented Jul 12, 2016

Test build #62161 has finished for PR 14151 at commit dafe981.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

put this in the proper package?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is currently in the same package as HadoopFileLineReader ? i.e. datasources. Should I move both of them to the package datasource.text ?

Copy link
Member Author

@ScrapCodes ScrapCodes Jul 13, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like they might get used in multiple other formats too, what do you intend by proper package is unclear to me.

@rxin
Copy link
Contributor

rxin commented Jul 12, 2016

BTW instead of a whole new format, I think this should just be an option in the existing text format.

@ScrapCodes ScrapCodes changed the title [SPARK-16496][SQL] Add wholetext as data source for SQL. [SPARK-16496][SQL] Add wholetext as option for reading text in SQL. Jul 13, 2016
@ScrapCodes
Copy link
Member Author

ScrapCodes commented Jul 13, 2016

Actually what you said sounds like a nice idea, I was considering is it possible to propagate this as an option in all other formats like CSV and Json too ?

@rxin
Copy link
Contributor

rxin commented Jul 13, 2016

For now let's just do it for text file. I took a look - I guess it is ok to leave them in datasources for now.

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from dafe981 to 6e83f46 Compare July 13, 2016 09:55
@SparkQA
Copy link

SparkQA commented Jul 13, 2016

Test build #62233 has finished for PR 14151 at commit 6e83f46.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) extends Iterator[Text]

@ScrapCodes
Copy link
Member Author

I have a question, should we keep a column with filenames ? in current approach we ignore key column.

@SparkQA
Copy link

SparkQA commented Jul 14, 2016

Test build #62308 has finished for PR 14151 at commit 82952e7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ScrapCodes
Copy link
Member Author

@rxin Do you think it looks okay now ?

@ScrapCodes
Copy link
Member Author

@rxin Ping !

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this really be a session-global configuration? It seems like something that is specific to a particular input file and should only be set when opening a given file.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea I don't think it should be a session wide config.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They are removed.

@SparkQA
Copy link

SparkQA commented Aug 16, 2016

Test build #63839 has finished for PR 14151 at commit 2540018.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ScrapCodes
Copy link
Member Author

@rxin Ping !

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Like what we did for csv and json, could you document this new option in DataFrameReader?

Copy link
Member

@HyukjinKwon HyukjinKwon Sep 30, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, we might need to document this within readwriter.py too.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good reminder ! @HyukjinKwon.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move it to TextOptions?

@ScrapCodes
Copy link
Member Author

Thanks @gatorsmile. I was actually wondering, where can I document this option.

@SparkQA
Copy link

SparkQA commented Sep 1, 2016

Test build #64773 has finished for PR 14151 at commit 8ac37c1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ScrapCodes
Copy link
Member Author

Hey @rxin, do you have further comments ?

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from 8ac37c1 to 74a5f28 Compare September 30, 2016 07:09
@SparkQA
Copy link

SparkQA commented Sep 30, 2016

Test build #66161 has finished for PR 14151 at commit 74a5f28.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) extends Iterator[Text]

@SparkQA
Copy link

SparkQA commented Sep 30, 2016

Test build #66164 has finished for PR 14151 at commit 3f8a177.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration)

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from 3f8a177 to e263b15 Compare October 5, 2016 05:38
@SparkQA
Copy link

SparkQA commented Oct 5, 2016

Test build #66375 has finished for PR 14151 at commit e263b15.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration)

@sameeragarwal
Copy link
Member

test this please

@SparkQA
Copy link

SparkQA commented Jun 16, 2017

Test build #78197 has finished for PR 14151 at commit e263b15.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration)

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from e263b15 to cab3323 Compare June 28, 2017 11:30
@ScrapCodes
Copy link
Member Author

@viirya Can you please take another look?

@gatorsmile
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Dec 6, 2017

Test build #84512 has finished for PR 14151 at commit da64f2d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@viirya viirya Dec 6, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: // scalastyle:on nonascii

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can avoid using var:

val reader = if (!wholeTextMode) {
  new HadoopFileLinesReader(file, confValue)
} else {
  new HadoopFileWholeTextReader(file, confValue)
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a doctest for wholetext too?

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from da64f2d to dd2ed3d Compare December 7, 2017 13:07
@SparkQA
Copy link

SparkQA commented Dec 7, 2017

Test build #84602 has finished for PR 14151 at commit dd2ed3d.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration)

@ScrapCodes
Copy link
Member Author

This python pydoc style is failing at [Row(value=u'hello\nthis')]. I could not find a way to fix it. Any help will be appreciated. It does not like the literal '\n'

[Row(value=u'hello'), Row(value=u'this')]
>>> df = spark.read.text('python/test_support/sql/text-test.txt', wholetext=True)
>>> df.collect()
[Row(value=u'hello\nthis')]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, can't we just do \\n?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That would fail the test, I suppose. I can give that a try though.

@SparkQA
Copy link

SparkQA commented Dec 8, 2017

Test build #84645 has finished for PR 14151 at commit 7e91020.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member

viirya commented Dec 8, 2017

retest this please.

@SparkQA
Copy link

SparkQA commented Dec 8, 2017

Test build #84648 has finished for PR 14151 at commit 7e91020.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@SparkQA
Copy link

SparkQA commented Dec 8, 2017

Test build #84652 has finished for PR 14151 at commit 7e91020.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member

viirya commented Dec 8, 2017

Looks the escaping is ok.

@gatorsmile
Copy link
Member

gatorsmile commented Dec 8, 2017

Since we expect users to use this one, instead of the RDD's wholeText reader. Could you add the new test cases from WholeTextFileRecordReaderSuite? Thanks!

@SparkQA
Copy link

SparkQA commented Dec 11, 2017

Test build #84704 has finished for PR 14151 at commit 66d5b45.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class WholeTextFileSuite extends QueryTest with SharedSQLContext

@ScrapCodes ScrapCodes force-pushed the SPARK-16496/wholetext branch from 989ab94 to 021039b Compare December 14, 2017 06:35
@SparkQA
Copy link

SparkQA commented Dec 14, 2017

Test build #84896 has finished for PR 14151 at commit 021039b.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@ScrapCodes
Copy link
Member Author

retest this please

@SparkQA
Copy link

SparkQA commented Dec 14, 2017

Test build #84905 has finished for PR 14151 at commit 021039b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

LGTM

@gatorsmile
Copy link
Member

Thanks! Merged to master.

The code style issues will be addressed by my other PRs.

@asfgit asfgit closed this in 40de176 Dec 14, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants