[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 <dhyun@apple.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
This commit is contained in:
parent
8ef4da753d
commit
614a5cc600
|
@ -319,6 +319,8 @@ public final class CalendarInterval implements Serializable {
|
||||||
appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond");
|
appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond");
|
||||||
rest %= MICROS_PER_MILLI;
|
rest %= MICROS_PER_MILLI;
|
||||||
appendUnit(sb, rest, "microsecond");
|
appendUnit(sb, rest, "microsecond");
|
||||||
|
} else if (months == 0) {
|
||||||
|
sb.append(" 0 microseconds");
|
||||||
}
|
}
|
||||||
|
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
|
|
|
@ -41,6 +41,9 @@ public class CalendarIntervalSuite {
|
||||||
public void toStringTest() {
|
public void toStringTest() {
|
||||||
CalendarInterval i;
|
CalendarInterval i;
|
||||||
|
|
||||||
|
i = new CalendarInterval(0, 0);
|
||||||
|
assertEquals("interval 0 microseconds", i.toString());
|
||||||
|
|
||||||
i = new CalendarInterval(34, 0);
|
i = new CalendarInterval(34, 0);
|
||||||
assertEquals("interval 2 years 10 months", i.toString());
|
assertEquals("interval 2 years 10 months", i.toString());
|
||||||
|
|
||||||
|
|
Loading…
Reference in a new issue