You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2019/05/07 18:09:50 UTC

[spark] branch branch-2.3 updated: [SPARK-27624][CORE] Fix CalenderInterval to show an empty interval correctly

This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 8eac465  [SPARK-27624][CORE] Fix CalenderInterval to show an empty interval correctly
8eac465 is described below

commit 8eac465059f95d02df9f0df4c992c28dac4d4f1e
Author: Dongjoon Hyun <dh...@apple.com>
AuthorDate: Tue May 7 11:08:55 2019 -0700

    [SPARK-27624][CORE] Fix CalenderInterval to show an empty interval correctly
    
    ## What changes were proposed in this pull request?
    
    If the interval is `0`, it doesn't show both the value `0` and the unit at all. For example, this happens in the explain plans and Spark Web UI on `EventTimeWatermark` diagram.
    
    **BEFORE**
    ```scala
    scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "1 microsecond").explain
    == Physical Plan ==
    EventTimeWatermark ts#0: timestamp, interval 1 microseconds
    +- StreamingRelation FileSource[/tmp/t], [ts#0]
    
    scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "0 microsecond").explain
    == Physical Plan ==
    EventTimeWatermark ts#3: timestamp, interval
    +- StreamingRelation FileSource[/tmp/t], [ts#3]
    ```
    
    **AFTER**
    ```scala
    scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "1 microsecond").explain
    == Physical Plan ==
    EventTimeWatermark ts#0: timestamp, interval 1 microseconds
    +- StreamingRelation FileSource[/tmp/t], [ts#0]
    
    scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "0 microsecond").explain
    == Physical Plan ==
    EventTimeWatermark ts#3: timestamp, interval 0 microseconds
    +- StreamingRelation FileSource[/tmp/t], [ts#3]
    ```
    
    ## How was this patch tested?
    
    Pass the Jenkins with the updated test case.
    
    Closes #24516 from dongjoon-hyun/SPARK-27624.
    
    Authored-by: Dongjoon Hyun <dh...@apple.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
    (cherry picked from commit 614a5cc600b0ac01c5d03b1dc5fdf996ef18ac0e)
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 .../src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java  | 2 ++
 .../test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java | 3 +++
 2 files changed, 5 insertions(+)

diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
index 621f2c6..611b2a2 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
@@ -319,6 +319,8 @@ public final class CalendarInterval implements Serializable {
       appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond");
       rest %= MICROS_PER_MILLI;
       appendUnit(sb, rest, "microsecond");
+    } else if (months == 0) {
+      sb.append(" 0 microseconds");
     }
 
     return sb.toString();
diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java
index 9e69e26..1e55691 100644
--- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java
+++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java
@@ -41,6 +41,9 @@ public class CalendarIntervalSuite {
   public void toStringTest() {
     CalendarInterval i;
 
+    i = new CalendarInterval(0, 0);
+    assertEquals("interval 0 microseconds", i.toString());
+
     i = new CalendarInterval(34, 0);
     assertEquals("interval 2 years 10 months", i.toString());
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org