Skip to content

Conversation

@maropu
Copy link
Member

@maropu maropu commented Apr 21, 2017

What changes were proposed in this pull request?

This pr supported a DDL-formatted string in DataFrameReader.schema.
This fix could make users easily define a schema without importing o.a.spark.sql.types._.

How was this patch tested?

Added tests in DataFrameReaderWriterSuite.

@SparkQA
Copy link

SparkQA commented Apr 21, 2017

Test build #76034 has finished for PR 17719 at commit 8447a6d.

  • This patch fails Python style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 21, 2017

Test build #76037 has finished for PR 17719 at commit a1a2e35.

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

@maropu
Copy link
Member Author

maropu commented Apr 22, 2017

cc: @gatorsmile

elif isinstance(schema, basestring):
self._jreader = self._jreader.schema(schema)
else:
raise TypeError("schema should be StructType")
Copy link
Member

Choose a reason for hiding this comment

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

Update this message?

Copy link
Member Author

Choose a reason for hiding this comment

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

yea, I'll do

*
* @since 2.3.0
*/
def schema(schemaString: String): DataFrameReader = {
Copy link
Member

Choose a reason for hiding this comment

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

This change will make PySpark API inconsistent with the Scala API

Copy link
Member Author

Choose a reason for hiding this comment

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

Sorry, but I probably missed your point. What's the API consistency you pointed out here?
I just made the python APIs the same with the Scala ones like:

--- python
>>> from pyspark.sql.types import *
>>> fields = [StructField('a', IntegerType(), True), StructField('b', StringType(), True), StructField('c', DoubleType(), True)]
>>> schema = StructType(fields)
>>> spark.read.schema(schema).csv("/Users/maropu/Desktop/test.csv").show()
+---+----+---+
|  a|   b|  c|
+---+----+---+
|  1| aaa|0.3|
+---+----+---+

>>> spark.read.schema("a INT, b STRING, c DOUBLE").csv("/Users/maropu/Desktop/test.csv").show()
+---+----+---+
|  a|   b|  c|
+---+----+---+
|  1| aaa|0.3|
+---+----+---+

--- scala
scala> import org.apache.spark.sql.types._
scala> fields = StructField("a", IntegerType) :: StructField("b", StringType) :: StructField("c", DoubleType) :: Nil
scala> val schema = StructType(fields)
scala> spark.read.schema(schema).csv("/Users/maropu/Desktop/test.csv").show
+---+----+---+
|  a|   b|  c|
+---+----+---+
|  1| aaa|0.3|
+---+----+---+

scala> spark.read.schema("a INT, b STRING, c DOUBLE").csv("/Users/maropu/Desktop/test.csv").show
+---+----+---+
|  a|   b|  c|
+---+----+---+
|  1| aaa|0.3|
+---+----+---+

Copy link
Member

Choose a reason for hiding this comment

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

Sorry, I misread the Python codes.

@SparkQA
Copy link

SparkQA commented Apr 22, 2017

Test build #76061 has finished for PR 17719 at commit 5fe5e39.

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

@maropu
Copy link
Member Author

maropu commented Apr 26, 2017

@gatorsmile ping

1 similar comment
@maropu
Copy link
Member Author

maropu commented May 10, 2017

@gatorsmile ping

inference step, and thus speed up data loading.
:param schema: a :class:`pyspark.sql.types.StructType` object
:param schema: a :class:`pyspark.sql.types.StructType` object or a DDL-formatted string
Copy link
Member

Choose a reason for hiding this comment

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

Could you give an example here to users?

Copy link
Member Author

Choose a reason for hiding this comment

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

ok

@gatorsmile
Copy link
Member

In PySpark, we have multiple ways to specify the schema. However, it sounds like we silently pick one of the input schemas. Could you submit a separate PR to fix the potential schema conflicts in PySpark?

Thanks!

@maropu
Copy link
Member Author

maropu commented May 11, 2017

Sure, I'd love to do tough, I probably missed your point. What's the scenario of the conflict you described? The current logic in readwriter.py just checks types, then decides a schema. Could you give me a concrete example? Thanks

@SparkQA
Copy link

SparkQA commented May 11, 2017

Test build #76773 has finished for PR 17719 at commit cc3afd7.

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

@SparkQA
Copy link

SparkQA commented May 11, 2017

Test build #76774 has finished for PR 17719 at commit 46994fb.

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

@gatorsmile
Copy link
Member

nvm. I will fix it later. Thanks!

@gatorsmile
Copy link
Member

LGTM

@gatorsmile
Copy link
Member

Thanks! Merging to master.

@asfgit asfgit closed this in 04901dd May 11, 2017
robert3005 pushed a commit to palantir/spark that referenced this pull request May 19, 2017
## What changes were proposed in this pull request?
This pr supported a DDL-formatted string in `DataFrameReader.schema`.
This fix could make users easily define a schema without importing  `o.a.spark.sql.types._`.

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite`.

Author: Takeshi Yamamuro <[email protected]>

Closes apache#17719 from maropu/SPARK-20431.
liyichao pushed a commit to liyichao/spark that referenced this pull request May 24, 2017
## What changes were proposed in this pull request?
This pr supported a DDL-formatted string in `DataFrameReader.schema`.
This fix could make users easily define a schema without importing  `o.a.spark.sql.types._`.

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite`.

Author: Takeshi Yamamuro <[email protected]>

Closes apache#17719 from maropu/SPARK-20431.
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.

3 participants