Skip to content
Snippets Groups Projects
Commit 7a00c658 authored by hyukjinkwon's avatar hyukjinkwon Committed by Shixiong Zhu
Browse files

[SPARK-21147][SS] Throws an analysis exception when a user-specified schema is...

[SPARK-21147][SS] Throws an analysis exception when a user-specified schema is given in socket/rate sources

## What changes were proposed in this pull request?

This PR proposes to throw an exception if a schema is provided by user to socket source as below:

**socket source**

```scala
import org.apache.spark.sql.types._

val userSpecifiedSchema = StructType(
  StructField("name", StringType) ::
  StructField("area", StringType) :: Nil)
val df = spark.readStream.format("socket").option("host", "localhost").option("port", 9999).schema(userSpecifiedSchema).load
df.printSchema
```

Before

```
root
 |-- value: string (nullable = true)
```

After

```
org.apache.spark.sql.AnalysisException: The socket source does not support a user-specified schema.;
  at org.apache.spark.sql.execution.streaming.TextSocketSourceProvider.sourceSchema(socket.scala:199)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
  at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
  at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
  ... 50 elided
```

**rate source**

```scala
spark.readStream.format("rate").schema(spark.range(1).schema).load().printSchema()
```

Before

```
root
 |-- timestamp: timestamp (nullable = true)
 |-- value: long (nullable = true)`
```

After

```
org.apache.spark.sql.AnalysisException: The rate source does not support a user-specified schema.;
  at org.apache.spark.sql.execution.streaming.RateSourceProvider.sourceSchema(RateSourceProvider.scala:57)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
  at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
  at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
  ... 48 elided
```

## How was this patch tested?

Unit test in `TextSocketStreamSuite` and `RateSourceSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18365 from HyukjinKwon/SPARK-21147.
parent ad459cfb
No related branches found
No related tags found
No related merge requests found
Loading
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment