[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-07-03 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199696129
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2163,9 +2163,9 @@ def json_tuple(col, *fields):
 @since(2.1)
 def from_json(col, schema, options={}):
 """
-Parses a column containing a JSON string into a :class:`MapType` with 
:class:`StringType`
-as keys type, :class:`StructType` or :class:`ArrayType` of 
:class:`StructType`\\s with
-the specified schema. Returns `null`, in the case of an unparseable 
string.
+Parses a column containing a JSON string into a :class:`MapType`, 
:class:`StructType`
+or :class:`ArrayType` of :class:`StructType`\\s with the specified 
schema. Returns
+`null`, in the case of an unparseable string.
--- End diff --

@viirya @MaxGekk does this generic solution in JacksonParser (see 
makeKeyConverter) address your concerns? 


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-07-01 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199356020
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2163,9 +2163,9 @@ def json_tuple(col, *fields):
 @since(2.1)
 def from_json(col, schema, options={}):
 """
-Parses a column containing a JSON string into a :class:`MapType` with 
:class:`StringType`
-as keys type, :class:`StructType` or :class:`ArrayType` of 
:class:`StructType`\\s with
-the specified schema. Returns `null`, in the case of an unparseable 
string.
+Parses a column containing a JSON string into a :class:`MapType`, 
:class:`StructType`
+or :class:`ArrayType` of :class:`StructType`\\s with the specified 
schema. Returns
+`null`, in the case of an unparseable string.
--- End diff --

For awareness, I also added Unit Tests for each of the supported key types:

```
[info] - SPARK-24682: roundtrip in to_json and from_json - Boolean as key 
(260 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Byte as key (277 
milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Short as key 
(210 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Integer as key 
(248 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Long as key (313 
milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Float as key 
(214 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Double as key 
(316 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - String as key 
(247 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Timestamp as key 
(284 milliseconds)
[info] - SPARK-24682: roundtrip in to_json and from_json - Date as key (179 
milliseconds)
```


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-07-01 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199350374
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2163,9 +2163,9 @@ def json_tuple(col, *fields):
 @since(2.1)
 def from_json(col, schema, options={}):
 """
-Parses a column containing a JSON string into a :class:`MapType` with 
:class:`StringType`
-as keys type, :class:`StructType` or :class:`ArrayType` of 
:class:`StructType`\\s with
-the specified schema. Returns `null`, in the case of an unparseable 
string.
+Parses a column containing a JSON string into a :class:`MapType`, 
:class:`StructType`
+or :class:`ArrayType` of :class:`StructType`\\s with the specified 
schema. Returns
+`null`, in the case of an unparseable string.
--- End diff --

I made the KeyConverter generic as proposed in the comments.  It now will 
support a Boolean as the key.  I don't think there is much value in having a 
Boolean (or Short, etc) as a key but I do think there is with dates.  I raised 
this issue as we have a Map which contains dates and account balances for each 
of those dates.  When we write it out to JSON with Spark we can't read it back 
in.


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-07-01 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199342794
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2163,9 +2163,9 @@ def json_tuple(col, *fields):
 @since(2.1)
 def from_json(col, schema, options={}):
 """
-Parses a column containing a JSON string into a :class:`MapType` with 
:class:`StringType`
-as keys type, :class:`StructType` or :class:`ArrayType` of 
:class:`StructType`\\s with
-the specified schema. Returns `null`, in the case of an unparseable 
string.
+Parses a column containing a JSON string into a :class:`MapType`, 
:class:`StructType`
+or :class:`ArrayType` of :class:`StructType`\\s with the specified 
schema. Returns
+`null`, in the case of an unparseable string.
--- End diff --

No, previously Dates and Times would not work.  If you put a Timestamp in 
the value of the key would be Long value of the time and when that would be 
read back you would get and exception saying String could not be converted to 
Long.

I just created the following test for a Boolean as a key and it fails.  I 
will make a generic solution for all key types with Unit Tests for each one.  
WIll report back soon.

java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String 
cannot be cast to java.lang.Boolean
'''
  test("roundtrip in to_json and from_json - Boolean as key") {
val df = Seq(Map(true -> 1)).toDF("map")
val optionsDate = Map("dateFormat" -> "dd/MM/")
val schema = MapType(BooleanType, IntegerType)
val readBack = df.select(to_json($"map").as("json"))
  .select(from_json($"json", schema).as("map"))
checkAnswer(df, readBack)
  }
'''


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-07-01 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199342210
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 ---
@@ -317,16 +292,52 @@ class JacksonParser(
 row
   }
 
+  private def parseTimestamp(stringValue: String): Long = {
+// This one will lose microseconds parts.
+// See https://issues.apache.org/jira/browse/SPARK-10681.x
+Try(options.timestampFormat.parse(stringValue).getTime * 1000L)
+  .getOrElse {
+// If it fails to parse, then tries the way used in 2.0 and 1.x 
for backwards
+// compatibility.
+DateTimeUtils.stringToTime(stringValue).getTime * 1000L
+  }
+  }
+
+  private def parseDate(stringValue: String): Int = {
+
Try(DateTimeUtils.millisToDays(options.dateFormat.parse(stringValue).getTime))
+  .orElse {
+// If it fails to parse, then tries the way used in 2.0 and 1.x 
for backwards
+// compatibility.
+
Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(stringValue).getTime))
+  }
+  .getOrElse {
+// In Spark 1.5.0, we store the data as number of days since epoch 
in string.
+// So, we just convert it to Int.
+stringValue.toInt
+  }
+  }
+
   /**
* Parse an object as a Map, preserving all fields.
*/
   private def convertMap(
   parser: JsonParser,
+  keyType: DataType,
   fieldConverter: ValueConverter): MapData = {
-val keys = ArrayBuffer.empty[UTF8String]
+val keys = ArrayBuffer.empty[Any]
 val values = ArrayBuffer.empty[Any]
 while (nextUntil(parser, JsonToken.END_OBJECT)) {
-  keys += UTF8String.fromString(parser.getCurrentName)
+
+  val keyValue = keyType match {
+case DateType =>
+  parseDate(parser.getCurrentName)
+case TimestampType =>
+  parseTimestamp(parser.getCurrentName)
+case _ =>
+  UTF8String.fromString(parser.getCurrentName)
+  }
+
+  keys += keyValue
--- End diff --

Hi @MaxGekk , @HyukjinKwon , my thoughts are that most basic types will be 
supported by virtue of being converted to a string and back to their correct 
type.  I could create Unit Tests to show which types work.  What I think will 
not work is nested types (e.g. a Seq of Ints) or perhaps a BinaryType, as I can 
imagine that being converted to string and back may not work well.  

I can add a generic way of implementing the code, as in the code snippet 
above, but I don't think other types will use it.  The real use case here is 
when we have to do some formatting when reading and writing the key value.  




---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-30 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199330240
  
--- Diff: python/pyspark/sql/functions.py ---
@@ -2163,9 +2163,9 @@ def json_tuple(col, *fields):
 @since(2.1)
 def from_json(col, schema, options={}):
 """
-Parses a column containing a JSON string into a :class:`MapType` with 
:class:`StringType`
-as keys type, :class:`StructType` or :class:`ArrayType` of 
:class:`StructType`\\s with
-the specified schema. Returns `null`, in the case of an unparseable 
string.
+Parses a column containing a JSON string into a :class:`MapType`, 
:class:`StructType`
+or :class:`ArrayType` of :class:`StructType`\\s with the specified 
schema. Returns
+`null`, in the case of an unparseable string.
--- End diff --

I think that other basic types (e.g Int, Long, etc) were already supported 
along with String.  As long as they can be directly converted from String to 
that type specified in the schema then they were ok.  What we have added here 
is parsing on the DateType and TimestampType.  I don't think complex types 
would ever be supported.  And I don't think other basic types need the parsing 
that date/times do.

Do you think we should specify which types are supported as keys?


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-30 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199330156
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -43,6 +43,11 @@ private[sql] class JacksonGenerator(
   // we can directly access data in `ArrayData` without the help of 
`SpecificMutableRow`.
   private type ValueWriter = (SpecializedGetters, Int) => Unit
 
+  // A `ValueReader` is responsible for reading a field of an 
`InternalRow` to a String.
+  // The only purpose of this is to read the key values for a map so that 
they can be
+  // written as JSON filed names.
--- End diff --

Fixed


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-30 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199319281
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -163,6 +175,22 @@ private[sql] class JacksonGenerator(
   s"with the type of $dataType to JSON.")
   }
 
+  private def makeReader(dataType: DataType): ValueReader = dataType match 
{
+
+case TimestampType =>
+  (row: SpecializedGetters, ordinal: Int) =>
+
options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal)))
+
+case DateType =>
+  (row: SpecializedGetters, ordinal: Int) =>
+
options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal)))
+
+case _ =>
+  (row: SpecializedGetters, ordinal: Int) =>
+row.get(ordinal, dataType).toString
--- End diff --

Actually, now that I consider it a bit more I think nested types are not 
possible.  The result has to be stored as a String, e.g.

"key": "value"

Putting the nested type inside the name of the JSON wouldn't be possible.  


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-30 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199319246
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 ---
@@ -163,6 +175,22 @@ private[sql] class JacksonGenerator(
   s"with the type of $dataType to JSON.")
   }
 
+  private def makeReader(dataType: DataType): ValueReader = dataType match 
{
+
+case TimestampType =>
+  (row: SpecializedGetters, ordinal: Int) =>
+
options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal)))
+
+case DateType =>
+  (row: SpecializedGetters, ordinal: Int) =>
+
options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal)))
+
+case _ =>
+  (row: SpecializedGetters, ordinal: Int) =>
+row.get(ordinal, dataType).toString
--- End diff --

I considered this the default case and thought it would be covered by 
existing Unit Tests.  Which types should I add tests for?  Int, String, Long, 
etc.  All of the children of AtomicType?

Nested types is interesting.  I will add some tests for keys with a 
Seq[Int], Map[String, Int] to see that they work.  If there was something else 
you were thinking of please let me know.


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-30 Thread patrickmcgloin
Github user patrickmcgloin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21671#discussion_r199316164
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 ---
@@ -317,16 +292,52 @@ class JacksonParser(
 row
   }
 
+  private def parseTimestamp(stringValue: String): Long = {
+// This one will lose microseconds parts.
+// See https://issues.apache.org/jira/browse/SPARK-10681.x
+Try(options.timestampFormat.parse(stringValue).getTime * 1000L)
+  .getOrElse {
+// If it fails to parse, then tries the way used in 2.0 and 1.x 
for backwards
+// compatibility.
+DateTimeUtils.stringToTime(stringValue).getTime * 1000L
+  }
+  }
+
+  private def parseDate(stringValue: String): Int = {
+
Try(DateTimeUtils.millisToDays(options.dateFormat.parse(stringValue).getTime))
+  .orElse {
+// If it fails to parse, then tries the way used in 2.0 and 1.x 
for backwards
+// compatibility.
+
Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(stringValue).getTime))
+  }
+  .getOrElse {
+// In Spark 1.5.0, we store the data as number of days since epoch 
in string.
+// So, we just convert it to Int.
+stringValue.toInt
+  }
+  }
+
   /**
* Parse an object as a Map, preserving all fields.
*/
   private def convertMap(
   parser: JsonParser,
+  keyType: DataType,
   fieldConverter: ValueConverter): MapData = {
-val keys = ArrayBuffer.empty[UTF8String]
+val keys = ArrayBuffer.empty[Any]
 val values = ArrayBuffer.empty[Any]
 while (nextUntil(parser, JsonToken.END_OBJECT)) {
-  keys += UTF8String.fromString(parser.getCurrentName)
+
+  val keyValue = keyType match {
+case DateType =>
+  parseDate(parser.getCurrentName)
+case TimestampType =>
+  parseTimestamp(parser.getCurrentName)
+case _ =>
+  UTF8String.fromString(parser.getCurrentName)
+  }
+
+  keys += keyValue
--- End diff --

I originally wanted to use the existing ValueConverter but obviously that 
did not work as that tries to get the value from the JSON (e.g. 
parser.getIntValue) and we need to get the name of the field 
(parser.getCurrentName).

I could create a new "KeyConverter".  It would look something like this.  
Shall I do that?
```
  private def parseJsonFieldName[R >: Null](parser: JsonParser)
   (f: PartialFunction[String, R]): 
R = {
f.apply(parser.getText)
  }

  private type KeyConverter = JsonParser => AnyRef
  def makeNameConverter(keyType: DataType): KeyConverter = keyType match {
case DateType =>
  (parser: JsonParser) => parseJsonFieldName[Int](parser) {
case str: String => parseDate(str)
  }
case TimestampType =>
  (parser: JsonParser) => parseJsonFieldName[Long](parser) {
case str: String => parseTimestamp(str)
  }
case _ =>
  (parser: JsonParser) => parseJsonFieldName[UTF8String](parser) {
case str: String => UTF8String.fromString(str)
  }
  }
```


---

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



[GitHub] spark pull request #21671: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-29 Thread patrickmcgloin
GitHub user patrickmcgloin opened a pull request:

https://github.com/apache/spark/pull/21671

[SPARK-24682] [SQL] from_json / to_json now handle java.sql.Date/Timestamp 
as Map key

## What changes were proposed in this pull request?

When generating JSON the key is not just converted to a String now, instead 
the type is checked and for Dates and Timestamps the formatting happens first 
(if provided). When reading JSON, if the Schema says that a Map has a Date or 
Timestamp as the key then once again the formatting is done before the String 
is converted to the correct value for a Date or Timestamp.

## How was this patch tested?

I have added Unit Tests.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/patrickmcgloin/spark DATES_IN_MAPS

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21671.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21671


commit 58576fabf7b3ba93c177c78356bb17492e8b3d80
Author: Patrick McGloin 
Date:   2018-03-30T14:09:25Z

SPARK-23565: Improved error message for when the number of sources for a 
query changes

commit 7e88efab0c1479c1bf1e2df42906873ed2f5e050
Author: Patrick McGloin 
Date:   2018-04-25T16:51:09Z

Only one space after the period / dot

commit b35acf543cc0af66a56081457c53ab450b4cdcdb
Author: Patrick McGloin 
Date:   2018-04-25T16:52:57Z

Removed UNT as not needed for logging change

commit 0d5ae1f70ad7d99079b84219f5a2da99ab1ff6fd
Author: Patrick McGloin 
Date:   2018-06-27T17:51:39Z

Merge remote-tracking branch 'upstream/master'

commit 9686807a9fc729de0b63f68395f3a9e688fb087f
Author: Patrick McGloin 
Date:   2018-06-29T14:41:30Z

Allow Dates and Timestamps to be keys in JSON Maps




---

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



[GitHub] spark issue #21670: [SPARK-24682] [SQL] from_json / to_json now handle java....

2018-06-29 Thread patrickmcgloin
Github user patrickmcgloin commented on the issue:

https://github.com/apache/spark/pull/21670
  
This is mistake.  I have not put the correct code changes with the pull 
request.  Will close.


---

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



[GitHub] spark pull request #21670: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-29 Thread patrickmcgloin
Github user patrickmcgloin closed the pull request at:

https://github.com/apache/spark/pull/21670


---

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



[GitHub] spark pull request #21670: [SPARK-24682] [SQL] from_json / to_json now handl...

2018-06-29 Thread patrickmcgloin
GitHub user patrickmcgloin opened a pull request:

https://github.com/apache/spark/pull/21670

[SPARK-24682] [SQL] from_json / to_json now handle java.sql.Date/Timestamp 
as Map key

## What changes were proposed in this pull request?

When generating JSON the key is not just converted to a String now, instead 
the type is checked and for Dates and Timestamps the formatting happens first 
(if provided).  When reading JSON, if the Schema says that a Map has a Date or 
Timestamp as the key then once again the formatting is done before the String 
is converted to the correct value for a Date or Timestamp. 

## How was this patch tested?

I have added Unit Tests.

Please review http://spark.apache.org/contributing.html before opening a 
pull request.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/patrickmcgloin/spark master

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21670.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21670


commit 58576fabf7b3ba93c177c78356bb17492e8b3d80
Author: Patrick McGloin 
Date:   2018-03-30T14:09:25Z

SPARK-23565: Improved error message for when the number of sources for a 
query changes

commit 7e88efab0c1479c1bf1e2df42906873ed2f5e050
Author: Patrick McGloin 
Date:   2018-04-25T16:51:09Z

Only one space after the period / dot

commit b35acf543cc0af66a56081457c53ab450b4cdcdb
Author: Patrick McGloin 
Date:   2018-04-25T16:52:57Z

Removed UNT as not needed for logging change




---

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



[GitHub] spark pull request #20946: [SPARK-23565] [SS] New error message for structur...

2018-04-25 Thread patrickmcgloin
Github user patrickmcgloin closed the pull request at:

https://github.com/apache/spark/pull/20946


---

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



[GitHub] spark pull request #20946: [SPARK-23565] [SS] New error message for structur...

2018-04-25 Thread patrickmcgloin
GitHub user patrickmcgloin reopened a pull request:

https://github.com/apache/spark/pull/20946

[SPARK-23565] [SS] New error message for structured streaming sources 
assertion

## What changes were proposed in this pull request?

A more informative message to tell you why a structured streaming query 
cannot continue if you have added more sources, than there are in the existing 
checkpoint offsets.

## How was this patch tested?

I added a Unit Test.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/patrickmcgloin/spark master

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/20946.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #20946


commit 58576fabf7b3ba93c177c78356bb17492e8b3d80
Author: Patrick McGloin <mcgloin.patrick@...>
Date:   2018-03-30T14:09:25Z

SPARK-23565: Improved error message for when the number of sources for a 
query changes

commit 7e88efab0c1479c1bf1e2df42906873ed2f5e050
Author: Patrick McGloin <mcgloin.patrick@...>
Date:   2018-04-25T16:51:09Z

Only one space after the period / dot

commit b35acf543cc0af66a56081457c53ab450b4cdcdb
Author: Patrick McGloin <mcgloin.patrick@...>
Date:   2018-04-25T16:52:57Z

Removed UNT as not needed for logging change




---

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



[GitHub] spark issue #20946: [SPARK-23565] [SS] New error message for structured stre...

2018-04-25 Thread patrickmcgloin
Github user patrickmcgloin commented on the issue:

https://github.com/apache/spark/pull/20946
  
Hi @xuanyuanking, please see comments above.


---

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



[GitHub] spark issue #20946: [SPARK-23565] [SS] New error message for structured stre...

2018-04-25 Thread patrickmcgloin
Github user patrickmcgloin commented on the issue:

https://github.com/apache/spark/pull/20946
  
Hi Xuan, 

I have updated the title, removed the extra white space after the period / 
dot and removed the UT.


---

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



[GitHub] spark pull request #20946: [SPARK-23565] [SQL] New error message for structu...

2018-03-30 Thread patrickmcgloin
GitHub user patrickmcgloin opened a pull request:

https://github.com/apache/spark/pull/20946

[SPARK-23565] [SQL] New error message for structured streaming sources 
assertion

## What changes were proposed in this pull request?

A more informative message to tell you why a structured streaming query 
cannot continue if you have added more sources, than there are in the existing 
checkpoint offsets.

## How was this patch tested?

I added a Unit Test.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/patrickmcgloin/spark master

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/20946.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #20946


commit 58576fabf7b3ba93c177c78356bb17492e8b3d80
Author: Patrick McGloin <mcgloin.patrick@...>
Date:   2018-03-30T14:09:25Z

SPARK-23565: Improved error message for when the number of sources for a 
query changes




---

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