From e018e84dd37146ebe443c42aa42c1b383a9a819b Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Mon, 1 Jul 2019 15:12:39 +0530 Subject: [PATCH 01/13] SPARK-24695: To add support to return Calender interval from udf. ## What changes were proposed in this pull request? This change adds capability to return Calender interval from udf. Earlier, the udf of Type (String => CalendarInterval) was throwing Exception stating: Schema for type org.apache.spark.unsafe.types.CalendarInterval is not supported java.lang.UnsupportedOperationException: Schema for type org.apache.spark.unsafe.types.CalendarInterval is not supported at org.apache.spark.sql.catalyst.ScalaReflection391anonfun.apply(ScalaReflection.scala:781) ## How was this patch tested? Added test case in ScalaReflectionSuite.scala and ExpressionEncoderSuite.scala Also, tested by creating an udf that returns Calendar interval. jira entry for detail: https://issues.apache.org/jira/browse/SPARK-24695 --- .../unsafe/types/CalendarIntervalSuite.java | 268 ------------------ .../expressions/SpecializedGetters.java | 4 +- .../catalyst/expressions/UnsafeArrayData.java | 1 - .../sql/catalyst/expressions/UnsafeRow.java | 1 - .../expressions/codegen/UnsafeWriter.java | 2 +- .../spark/sql}/types/CalendarInterval.java | 5 +- .../spark/sql/vectorized/ColumnVector.java | 2 +- .../spark/sql/vectorized/ColumnarArray.java | 1 - .../spark/sql/vectorized/ColumnarBatch.java | 1 - .../spark/sql/vectorized/ColumnarRow.java | 1 - .../spark/sql/catalyst/ScalaReflection.scala | 4 +- .../analysis/StreamingJoinHelper.scala | 2 - .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/JoinedRow.scala | 2 +- .../sql/catalyst/expressions/TimeWindow.scala | 1 - .../sql/catalyst/expressions/arithmetic.scala | 1 - .../expressions/codegen/CodeGenerator.scala | 2 +- .../expressions/collectionOperations.scala | 1 - .../expressions/datetimeExpressions.scala | 2 +- .../spark/sql/catalyst/expressions/hash.scala | 2 +- .../sql/catalyst/expressions/literals.scala | 3 +- .../spark/sql/catalyst/expressions/rows.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/EventTimeWatermark.scala | 2 +- .../sql/catalyst/util/GenericArrayData.scala | 4 +- .../sql/types/CalendarIntervalSuite.java | 268 ++++++++++++++++++ .../spark/sql/RandomDataGenerator.scala | 1 - .../sql/catalyst/ScalaReflectionSuite.scala | 33 ++- .../catalyst/analysis/TypeCoercionSuite.scala | 1 - .../analysis/UnsupportedOperationsSuite.scala | 3 +- .../encoders/ExpressionEncoderSuite.scala | 20 +- .../sql/catalyst/expressions/CastSuite.scala | 2 +- .../CollectionExpressionsSuite.scala | 1 - .../expressions/DateExpressionsSuite.scala | 2 +- .../expressions/HashExpressionsSuite.scala | 2 +- .../expressions/LiteralExpressionSuite.scala | 2 - .../expressions/LiteralGenerator.scala | 1 - .../expressions/MutableProjectionSuite.scala | 1 - .../expressions/ObjectExpressionsSuite.scala | 2 +- .../expressions/UnsafeRowConverterSuite.scala | 2 +- .../GenerateUnsafeProjectionSuite.scala | 2 +- .../optimizer/FilterPushdownSuite.scala | 2 +- .../LeftSemiAntiJoinPushDownSuite.scala | 2 +- .../parser/ExpressionParserSuite.scala | 1 - .../sql/catalyst/util/UnsafeArraySuite.scala | 2 +- .../vectorized/ColumnVectorUtils.java | 2 +- .../vectorized/MutableColumnarRow.java | 1 - .../scala/org/apache/spark/sql/Dataset.scala | 1 - .../streaming/EventTimeWatermarkExec.scala | 2 +- .../execution/streaming/GroupStateImpl.scala | 2 +- .../StreamingSymmetricHashJoinHelper.scala | 1 - .../continuous/ContinuousTrigger.scala | 2 +- .../org/apache/spark/sql/functions.scala | 10 +- .../spark/sql/streaming/ProcessingTime.scala | 2 +- .../apache/spark/sql/DateFunctionsSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../vectorized/ColumnarBatchSuite.scala | 2 +- .../catalyst/ExpressionSQLBuilderSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 1 - 59 files changed, 347 insertions(+), 357 deletions(-) delete mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java rename {common/unsafe/src/main/java/org/apache/spark/unsafe => sql/catalyst/src/main/java/org/apache/spark/sql}/types/CalendarInterval.java (98%) create mode 100644 sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java 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 deleted file mode 100644 index 994af8f08244..000000000000 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ /dev/null @@ -1,268 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.unsafe.types; - -import org.junit.Test; - -import static org.junit.Assert.*; -import static org.apache.spark.unsafe.types.CalendarInterval.*; - -public class CalendarIntervalSuite { - - @Test - public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); - - assertNotSame(i1, i2); - assertNotSame(i1, i3); - assertNotSame(i2, i3); - assertEquals(i1, i4); - } - - @Test - 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()); - - i = new CalendarInterval(-34, 0); - assertEquals("interval -2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); - - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); - - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); - } - - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - String input; - - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); - - input = "interval -5 years 23 month "; - assertEquals(fromString(input), result); - - input = " interval -5 years 23 month "; - assertEquals(fromString(input), result); - - // Error cases - input = "interval 3month 1 hour"; - assertNull(fromString(input)); - - input = "interval 3 moth 1 hour"; - assertNull(fromString(input)); - - input = "interval"; - assertNull(fromString(input)); - - input = "int"; - assertNull(fromString(input)); - - input = ""; - assertNull(fromString(input)); - - input = null; - assertNull(fromString(input)); - } - - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("cannot be null or blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid interval")); - } - } - } - - @Test - public void fromYearMonthStringTest() { - String input; - CalendarInterval i; - - input = "99-10"; - i = new CalendarInterval(99 * 12 + 10, 0L); - assertEquals(fromYearMonthString(input), i); - - input = "-8-10"; - i = new CalendarInterval(-8 * 12 - 10, 0L); - assertEquals(fromYearMonthString(input), i); - - try { - input = "99-15"; - fromYearMonthString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("month 15 outside range")); - } - } - - @Test - public void fromDayTimeStringTest() { - String input; - CalendarInterval i; - - input = "5 12:40:30.999999999"; - i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); - assertEquals(fromDayTimeString(input), i); - - input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); - assertEquals(fromDayTimeString(input), i); - - input = "-3 0:0:0"; - i = new CalendarInterval(0, -3 * MICROS_PER_DAY); - assertEquals(fromDayTimeString(input), i); - - try { - input = "5 30:12:20"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("hour 30 outside range")); - } - - try { - input = "5 30-12"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("not match day-time format")); - } - } - - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - - @Test - public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); - } - - @Test - public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - } -} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java index eea7149d0259..57190d4a4aa8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java @@ -19,10 +19,10 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.CalendarInterval; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; public interface SpecializedGetters { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index db6401b18c0e..cd1f8fc17363 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -36,7 +36,6 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 8fd6029e976e..965bf5ff2e2f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -37,7 +37,6 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; import static org.apache.spark.sql.types.DataTypes.*; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 95263a0da95a..3762919a63d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -19,10 +19,10 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; import org.apache.spark.sql.catalyst.expressions.UnsafeMapData; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.types.CalendarInterval; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java similarity index 98% rename from common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java index 15d3e8cdfbad..787efe70626f 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.unsafe.types; +package org.apache.spark.sql.types; import java.io.Serializable; import java.util.Locale; @@ -103,8 +103,7 @@ public static CalendarInterval fromCaseInsensitiveString(String s) { throw new IllegalArgumentException("Interval cannot be null or blank."); } String sInLowerCase = s.trim().toLowerCase(Locale.ROOT); - String interval = - sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; + String interval = sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; CalendarInterval cal = fromString(interval); if (cal == null) { throw new IllegalArgumentException("Invalid interval: " + s); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index f18d00359c90..a6035bb0120c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java @@ -17,9 +17,9 @@ package org.apache.spark.sql.vectorized; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.types.CalendarInterval; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java index 147dd24e16ce..007343f4c9b1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java index a2feac869ece..8e5334924f3b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index 4b9d3c5f5991..7d7b88213715 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -20,7 +20,6 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 2a4e84241a94..7607af71e3e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, _} import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** @@ -671,6 +671,8 @@ object ScalaReflection extends ScalaReflection { Schema(TimestampType, nullable = true) case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => Schema(DateType, nullable = true) case t if isSubtype(t, localTypeOf[java.sql.Date]) => Schema(DateType, nullable = true) + case t if isSubtype(t, localTypeOf[CalendarInterval]) => + Schema(CalendarIntervalType, nullable = true) case t if isSubtype(t, localTypeOf[BigDecimal]) => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index c1d72f9b58a4..69eeb9f3e5b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -25,8 +25,6 @@ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval - /** * Helper object for stream joins. See [[StreamingSymmetricHashJoinExec]] in SQL for more details. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 969128838eba..f505443e04b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} object Cast { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala index 7770684a5b39..adf6ea6e6fd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** * A mutable wrapper that makes two rows appear as a single concatenated row. Designed to diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 9aae678deb4b..b760abeb5003 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( timeColumn: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 22b29c3000c1..305d0373f0a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval @ExpressionDescription( usage = "_FUNC_(expr) - Returns the negated value of `expr`.", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 95fad412002e..9081b88163bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.{ParentClassLoader, Utils} /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 8477e63135e3..b0a3db1e85f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -34,7 +34,6 @@ import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, UTF8String} -import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.collection.OpenHashSet /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index ccf6b36effa0..fd1dbc8e709b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** * Common base class for time zone aware expressions. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 2aa1e6cc518c..8dfc9517bd40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the expressions for hashing. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 9cef3ecadc54..d01140e6a836 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -45,7 +45,8 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types._ + +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils object Literal { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 65539a2f00e6..edf4d60cae57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** * An extended version of [[InternalRow]] that implements all special getters, toString diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f08cb2a780fe..e5b547082155 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStat import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.RandomSampler /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala index 8441c2c481ec..6b15a834c536 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.plans.logical import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.types.CalendarInterval import org.apache.spark.sql.types.MetadataBuilder -import org.apache.spark.unsafe.types.CalendarInterval object EventTimeWatermark { /** The [[org.apache.spark.sql.types.Metadata]] key used to hold the eventTime watermark delay. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 83ad08d8e175..3e7d49f1513a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.util import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.types.{DataType, Decimal} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.sql.types.{CalendarInterval, DataType, Decimal} +import org.apache.spark.unsafe.types.UTF8String private object GenericArrayData { diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java new file mode 100644 index 000000000000..94bdb7473329 --- /dev/null +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java @@ -0,0 +1,268 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.types; + +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.apache.spark.sql.types.CalendarInterval.*; + +public class CalendarIntervalSuite { + + @Test + public void equalsTest() { + CalendarInterval i1 = new CalendarInterval(3, 123); + CalendarInterval i2 = new CalendarInterval(3, 321); + CalendarInterval i3 = new CalendarInterval(1, 123); + CalendarInterval i4 = new CalendarInterval(3, 123); + + assertNotSame(i1, i2); + assertNotSame(i1, i3); + assertNotSame(i2, i3); + assertEquals(i1, i4); + } + + @Test + 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()); + + i = new CalendarInterval(-34, 0); + assertEquals("interval -2 years -10 months", i.toString()); + + i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); + + i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); + assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + + i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); + } + + @Test + public void fromStringTest() { + testSingleUnit("year", 3, 36, 0); + testSingleUnit("month", 3, 3, 0); + testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); + testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); + testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); + testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); + testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); + testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); + testSingleUnit("microsecond", 3, 0, 3); + + String input; + + input = "interval -5 years 23 month"; + CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); + assertEquals(fromString(input), result); + + input = "interval -5 years 23 month "; + assertEquals(fromString(input), result); + + input = " interval -5 years 23 month "; + assertEquals(fromString(input), result); + + // Error cases + input = "interval 3month 1 hour"; + assertNull(fromString(input)); + + input = "interval 3 moth 1 hour"; + assertNull(fromString(input)); + + input = "interval"; + assertNull(fromString(input)); + + input = "int"; + assertNull(fromString(input)); + + input = ""; + assertNull(fromString(input)); + + input = null; + assertNull(fromString(input)); + } + + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); + } + + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } + } + + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + + @Test + public void fromYearMonthStringTest() { + String input; + CalendarInterval i; + + input = "99-10"; + i = new CalendarInterval(99 * 12 + 10, 0L); + assertEquals(fromYearMonthString(input), i); + + input = "-8-10"; + i = new CalendarInterval(-8 * 12 - 10, 0L); + assertEquals(fromYearMonthString(input), i); + + try { + input = "99-15"; + fromYearMonthString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("month 15 outside range")); + } + } + + @Test + public void fromDayTimeStringTest() { + String input; + CalendarInterval i; + + input = "5 12:40:30.999999999"; + i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); + assertEquals(fromDayTimeString(input), i); + + input = "10 0:12:0.888"; + i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); + assertEquals(fromDayTimeString(input), i); + + input = "-3 0:0:0"; + i = new CalendarInterval(0, -3 * MICROS_PER_DAY); + assertEquals(fromDayTimeString(input), i); + + try { + input = "5 30:12:20"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("hour 30 outside range")); + } + + try { + input = "5 30-12"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("not match day-time format")); + } + } + + @Test + public void fromSingleUnitStringTest() { + String input; + CalendarInterval i; + + input = "12"; + i = new CalendarInterval(12 * 12, 0L); + assertEquals(fromSingleUnitString("year", input), i); + + input = "100"; + i = new CalendarInterval(0, 100 * MICROS_PER_DAY); + assertEquals(fromSingleUnitString("day", input), i); + + input = "1999.38888"; + i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); + assertEquals(fromSingleUnitString("second", input), i); + + try { + input = String.valueOf(Integer.MAX_VALUE); + fromSingleUnitString("year", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + + try { + input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); + fromSingleUnitString("hour", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + } + + @Test + public void addTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + } + + @Test + public void subtractTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + } + + private static void testSingleUnit(String unit, int number, int months, long microseconds) { + String input1 = "interval " + number + " " + unit; + String input2 = "interval " + number + " " + unit + "s"; + CalendarInterval result = new CalendarInterval(months, microseconds); + assertEquals(fromString(input1), result); + assertEquals(fromString(input2), result); + } +} \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index d361e6248e2f..17fb6c88bcce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -25,7 +25,6 @@ import scala.util.Random import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval /** * Random data generators for Spark SQL DataTypes. These generators do not generate uniformly random diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index a2e58c3eaa0b..1412a8fc4da1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -34,7 +34,8 @@ case class PrimitiveData( floatField: Float, shortField: Short, byteField: Byte, - booleanField: Boolean) + booleanField: Boolean, + intervalField: CalendarInterval) case class NullableData( intField: java.lang.Integer, @@ -48,7 +49,8 @@ case class NullableData( decimalField: java.math.BigDecimal, dateField: Date, timestampField: Timestamp, - binaryField: Array[Byte]) + binaryField: Array[Byte], + intervalField: CalendarInterval) case class OptionalData( intField: Option[Int], @@ -58,7 +60,8 @@ case class OptionalData( shortField: Option[Short], byteField: Option[Byte], booleanField: Option[Boolean], - structField: Option[PrimitiveData]) + structField: Option[PrimitiveData], + intervalField: Option[CalendarInterval]) case class ComplexData( arrayField: Seq[Int], @@ -174,7 +177,8 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("shortField", ShortType, nullable = false), StructField("byteField", ByteType, nullable = false), StructField("booleanField", BooleanType, nullable = false))), - nullable = true)) + nullable = true), + StructField("intervalField", CalendarIntervalType, nullable = false)) } test("nullable data") { @@ -192,7 +196,8 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("decimalField", DecimalType.SYSTEM_DEFAULT, nullable = true), StructField("dateField", DateType, nullable = true), StructField("timestampField", TimestampType, nullable = true), - StructField("binaryField", BinaryType, nullable = true))), + StructField("binaryField", BinaryType, nullable = true), + StructField("intervalField", CalendarIntervalType, nullable = true))), nullable = true)) } @@ -207,7 +212,9 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("shortField", ShortType, nullable = true), StructField("byteField", ByteType, nullable = true), StructField("booleanField", BooleanType, nullable = true), - StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true))), + StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true), + StructField("intervalField", CalendarIntervalType, nullable = true) + )), nullable = true)) } @@ -248,7 +255,8 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("floatField", FloatType, nullable = false), StructField("shortField", ShortType, nullable = false), StructField("byteField", ByteType, nullable = false), - StructField("booleanField", BooleanType, nullable = false))), + StructField("booleanField", BooleanType, nullable = false), + StructField("intervalField", CalendarIntervalType, nullable = false))), nullable = true), StructField( "nestedArrayField", @@ -278,19 +286,20 @@ class ScalaReflectionSuite extends SparkFunSuite { } test("convert PrimitiveData to catalyst") { - val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) - val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) + val data = PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)) + val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true, + new CalendarInterval(2, 0)) val dataType = schemaFor[PrimitiveData].dataType assert(CatalystTypeConverters.createToCatalystConverter(dataType)(data) === convertedData) } test("convert Option[Product] to catalyst") { - val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) + val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)) val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), - Some(primitiveData)) + Some(primitiveData), Some(new CalendarInterval(2, 0))) val dataType = schemaFor[OptionalData].dataType val convertedData = InternalRow(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, - InternalRow(1, 1, 1, 1, 1, 1, true)) + InternalRow(1, 1, 1, 1, 1, 1, true), new CalendarInterval(2, 0)) assert(CatalystTypeConverters.createToCatalystConverter(dataType)(data) === convertedData) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 0b168d060ef6..200b013c6f41 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval class TypeCoercionSuite extends AnalysisTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 28a164b5d0ca..15dfd0300a27 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{FlatMapGroupsWithState, _} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.streaming.OutputMode -import org.apache.spark.sql.types.{IntegerType, LongType, MetadataBuilder} -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.{CalendarInterval, IntegerType, LongType, MetadataBuilder} /** A dummy command for testing unsupported operations. */ case class DummyCommand() extends Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 86e43d71e460..0cb76e8f9cfc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -202,13 +202,14 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Array(Option(InnerClass(1))), "array of optional inner class") - productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true)) + productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))) productTest( OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), - Some(PrimitiveData(1, 1, 1, 1, 1, 1, true)))) + Some(PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))), + Some(new CalendarInterval(2, 0)))) - productTest(OptionalData(None, None, None, None, None, None, None, None)) + productTest(OptionalData(None, None, None, None, None, None, None, None, None)) encodeDecodeTest(Seq(Some(1), None), "Option in array") encodeDecodeTest(Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), "Option in map") @@ -217,9 +218,10 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest(BoxedData(null, null, null, null, null, null, null)) - productTest(RepeatedStruct(PrimitiveData(1, 1, 1, 1, 1, 1, true) :: Nil)) + productTest(RepeatedStruct(PrimitiveData(1, 1, 1, 1, 1, 1, true, + new CalendarInterval(2, 0)) :: Nil)) - productTest((1, "test", PrimitiveData(1, 1, 1, 1, 1, 1, true))) + productTest((1, "test", PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)))) productTest( RepeatedData( @@ -227,7 +229,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes Seq(Integer.valueOf(1), null, Integer.valueOf(2)), Map(1 -> 2L), Map(1 -> null), - PrimitiveData(1, 1, 1, 1, 1, 1, true))) + PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)))) productTest(NestedArray(Array(Array(1, -2, 3), null, Array(4, 5, -6)))) @@ -275,17 +277,17 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[Long])) encodeDecodeTest( - (PrimitiveData(1, 1, 1, 1, 1, 1, true), (3, 30L)), + (PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)), (3, 30L)), "tuple with 2 product encoders")( ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[(Int, Long)])) encodeDecodeTest( - (PrimitiveData(1, 1, 1, 1, 1, 1, true), 3), + (PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)), 3), "tuple with flat encoder and product encoder")( ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[Int])) encodeDecodeTest( - (3, PrimitiveData(1, 1, 1, 1, 1, 1, true)), + (3, PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))), "tuple with product encoder and flat encoder")( ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[PrimitiveData])) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 4d667fd61ae0..228317dca935 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -806,7 +806,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast between string and interval") { - import org.apache.spark.unsafe.types.CalendarInterval + import org.apache.spark.sql.types.CalendarInterval checkEvaluation(Cast(Literal(""), CalendarIntervalType), null) checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 603073b40d7a..03282cc6f24d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH -import org.apache.spark.unsafe.types.CalendarInterval class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 04bb61a7486e..24d4ccff7f26 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index b5cfaf8f4b0f..125cd6c7ce0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.types.{ArrayType, StructType, _} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 269f1a09ac53..ae999c847c39 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -30,8 +30,6 @@ import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval - class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index be5fdb5b42ea..7a0c589621f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -23,7 +23,6 @@ import org.scalacheck.{Arbitrary, Gen} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval /** * Property is a high-level specification of behavior that should hold for a range of data points. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 0d594eb10962..26f998707d2c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index b6ca52f1d967..ded9eb7b8daf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, Generic import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class InvokeTargetClass extends Serializable { def filterInt(e: Any): Any = e.asInstanceOf[Int] > 0 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 69523fa81bc6..2ef124dc9a64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types.{IntegerType, LongType, _} import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase with ExpressionEvalHelper { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala index 01aa3579aea9..4a60769d6271 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class GenerateUnsafeProjectionSuite extends SparkFunSuite { test("Test unsafe projection string access pattern") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index cf4e9fcea2c6..4ecd8ed8212c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.CalendarInterval import org.apache.spark.sql.types.IntegerType -import org.apache.spark.unsafe.types.CalendarInterval class FilterPushdownSuite extends PlanTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index 00709adaa7c8..8c08772e11ab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.CalendarInterval import org.apache.spark.sql.types.IntegerType -import org.apache.spark.unsafe.types.CalendarInterval class LeftSemiPushdownSuite extends PlanTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index e16262ddb9cd..9d2c520969f4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval /** * Test basic expression parsing. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 61ce63faf0d2..19ddd9efb54c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class UnsafeArraySuite extends SparkFunSuite { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index 829f3ce750fe..a436c8555b2a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -33,7 +33,7 @@ import org.apache.spark.sql.vectorized.ColumnarArray; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.sql.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index fca7e3685912..c7063ef127e6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -26,7 +26,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.sql.vectorized.ColumnarRow; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c7a38710fa28..f40971edc8cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -57,7 +57,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.Utils private[sql] object Dataset { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala index 6d1131e6939d..fe0057c6dbe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.CalendarInterval import org.apache.spark.sql.types.MetadataBuilder -import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.AccumulatorV2 /** Class for collecting event time stats with an accumulator */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index dda9d41f630e..03fd3d197437 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout} import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala index 2d4c3c10e644..c82aee1b779f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming.WatermarkSupport.watermarkExpression import org.apache.spark.sql.execution.streaming.state.{StateStoreCoordinatorRef, StateStoreProvider, StateStoreProviderId} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala index bd343f380603..dc50afcf1cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala @@ -23,7 +23,7 @@ import scala.concurrent.duration.Duration import org.apache.spark.annotation.Evolving import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval /** * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 5fa3fd0a37a6..d72d02a5b25a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3086,14 +3086,14 @@ object functions { * @param timeColumn The column or the expression to use as the timestamp for windowing by time. * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for * valid duration identifiers. Note that the duration is a fixed length of * time, and does not vary over time according to a calendar. For example, * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check - * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration + * `org.apache.spark.sql.types.CalendarInterval` for valid duration * identifiers. This duration is likewise absolute, and does not vary * according to a calendar. * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start @@ -3142,14 +3142,14 @@ object functions { * @param timeColumn The column or the expression to use as the timestamp for windowing by time. * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for * valid duration identifiers. Note that the duration is a fixed length of * time, and does not vary over time according to a calendar. For example, * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check - * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration + * `org.apache.spark.sql.types.CalendarInterval` for valid duration * identifiers. This duration is likewise absolute, and does not vary * according to a calendar. * @@ -3187,7 +3187,7 @@ object functions { * @param timeColumn The column or the expression to use as the timestamp for windowing by time. * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, - * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for * valid duration identifiers. * * @group datetime_funcs diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala index 417d698bdbb0..3860bef8b886 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration import org.apache.spark.annotation.Evolving -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval /** * A trigger that runs a query periodically based on the processing time. If `interval` is 0, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 3f91b91850e8..c0ee65780cef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval class DateFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index ba1ac654c1a0..a581ee2d9c96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1469,7 +1469,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8753: add interval type") { - import org.apache.spark.unsafe.types.CalendarInterval + import org.apache.spark.sql.types.CalendarInterval val df = sql("select interval 3 years -3 month 7 week 123 microseconds") checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * 1000 * 1000 * 3600 * 24 * 7 + 123 ))) @@ -1494,8 +1494,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8945: add and subtract expressions for interval type") { - import org.apache.spark.unsafe.types.CalendarInterval - import org.apache.spark.unsafe.types.CalendarInterval.MICROS_PER_WEEK + import org.apache.spark.sql.types.CalendarInterval + import org.apache.spark.sql.types.CalendarInterval.MICROS_PER_WEEK val df = sql("select interval 3 years -3 month 7 week 123 microseconds as i") checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * MICROS_PER_WEEK + 123))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 758780c80b28..2cda197f8558 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class ColumnarBatchSuite extends SparkFunSuite { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index ae701f266bf4..1c6b0f525f95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { protected def checkSQL(e: Expression, expectedSQL: String): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index acfb84ede7ad..9a32836ac748 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -42,7 +42,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) From bdb800ed9e3966762a35e9256b49b21bb1c82dba Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:23:15 +0530 Subject: [PATCH 02/13] SPARK-24695: To incorporate review comments --- .../sql/types/CalendarIntervalSuite.java | 434 +++++++++--------- 1 file changed, 217 insertions(+), 217 deletions(-) diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java index 94bdb7473329..317d074877c3 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java @@ -1,18 +1,18 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. */ package org.apache.spark.sql.types; @@ -24,245 +24,245 @@ public class CalendarIntervalSuite { - @Test - public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); - - assertNotSame(i1, i2); - assertNotSame(i1, i3); - assertNotSame(i2, i3); - assertEquals(i1, i4); - } + @Test + public void equalsTest() { + CalendarInterval i1 = new CalendarInterval(3, 123); + CalendarInterval i2 = new CalendarInterval(3, 321); + CalendarInterval i3 = new CalendarInterval(1, 123); + CalendarInterval i4 = new CalendarInterval(3, 123); - @Test - public void toStringTest() { - CalendarInterval i; + assertNotSame(i1, i2); + assertNotSame(i1, i3); + assertNotSame(i2, i3); + assertEquals(i1, i4); + } - i = new CalendarInterval(0, 0); - assertEquals("interval 0 microseconds", i.toString()); + @Test + public void toStringTest() { + CalendarInterval i; - i = new CalendarInterval(34, 0); - assertEquals("interval 2 years 10 months", i.toString()); + i = new CalendarInterval(0, 0); + assertEquals("interval 0 microseconds", i.toString()); - i = new CalendarInterval(-34, 0); - assertEquals("interval -2 years -10 months", i.toString()); + i = new CalendarInterval(34, 0); + assertEquals("interval 2 years 10 months", i.toString()); - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); + i = new CalendarInterval(-34, 0); + assertEquals("interval -2 years -10 months", i.toString()); - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); - } + i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); + assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + + i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); + } - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); + @Test + public void fromStringTest() { + testSingleUnit("year", 3, 36, 0); + testSingleUnit("month", 3, 3, 0); + testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); + testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); + testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); + testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); + testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); + testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); + testSingleUnit("microsecond", 3, 0, 3); - String input; + String input; - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); + input = "interval -5 years 23 month"; + CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); + assertEquals(fromString(input), result); - input = "interval -5 years 23 month "; - assertEquals(fromString(input), result); + input = "interval -5 years 23 month "; + assertEquals(fromString(input), result); - input = " interval -5 years 23 month "; - assertEquals(fromString(input), result); + input = " interval -5 years 23 month "; + assertEquals(fromString(input), result); - // Error cases - input = "interval 3month 1 hour"; - assertNull(fromString(input)); + // Error cases + input = "interval 3month 1 hour"; + assertNull(fromString(input)); - input = "interval 3 moth 1 hour"; - assertNull(fromString(input)); + input = "interval 3 moth 1 hour"; + assertNull(fromString(input)); - input = "interval"; - assertNull(fromString(input)); + input = "interval"; + assertNull(fromString(input)); - input = "int"; - assertNull(fromString(input)); + input = "int"; + assertNull(fromString(input)); - input = ""; - assertNull(fromString(input)); + input = ""; + assertNull(fromString(input)); - input = null; - assertNull(fromString(input)); + input = null; + assertNull(fromString(input)); + } + + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); } - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("cannot be null or blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid interval")); - } - } + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } } - @Test - public void fromYearMonthStringTest() { - String input; - CalendarInterval i; - - input = "99-10"; - i = new CalendarInterval(99 * 12 + 10, 0L); - assertEquals(fromYearMonthString(input), i); - - input = "-8-10"; - i = new CalendarInterval(-8 * 12 - 10, 0L); - assertEquals(fromYearMonthString(input), i); - - try { - input = "99-15"; - fromYearMonthString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("month 15 outside range")); - } + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + + @Test + public void fromYearMonthStringTest() { + String input; + CalendarInterval i; + + input = "99-10"; + i = new CalendarInterval(99 * 12 + 10, 0L); + assertEquals(fromYearMonthString(input), i); + + input = "-8-10"; + i = new CalendarInterval(-8 * 12 - 10, 0L); + assertEquals(fromYearMonthString(input), i); + + try { + input = "99-15"; + fromYearMonthString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("month 15 outside range")); + } + } + + @Test + public void fromDayTimeStringTest() { + String input; + CalendarInterval i; + + input = "5 12:40:30.999999999"; + i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); + assertEquals(fromDayTimeString(input), i); + + input = "10 0:12:0.888"; + i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); + assertEquals(fromDayTimeString(input), i); + + input = "-3 0:0:0"; + i = new CalendarInterval(0, -3 * MICROS_PER_DAY); + assertEquals(fromDayTimeString(input), i); + + try { + input = "5 30:12:20"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("hour 30 outside range")); } - @Test - public void fromDayTimeStringTest() { - String input; - CalendarInterval i; - - input = "5 12:40:30.999999999"; - i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); - assertEquals(fromDayTimeString(input), i); - - input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); - assertEquals(fromDayTimeString(input), i); - - input = "-3 0:0:0"; - i = new CalendarInterval(0, -3 * MICROS_PER_DAY); - assertEquals(fromDayTimeString(input), i); - - try { - input = "5 30:12:20"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("hour 30 outside range")); - } - - try { - input = "5 30-12"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("not match day-time format")); - } + try { + input = "5 30-12"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("not match day-time format")); + } + } + + @Test + public void fromSingleUnitStringTest() { + String input; + CalendarInterval i; + + input = "12"; + i = new CalendarInterval(12 * 12, 0L); + assertEquals(fromSingleUnitString("year", input), i); + + input = "100"; + i = new CalendarInterval(0, 100 * MICROS_PER_DAY); + assertEquals(fromSingleUnitString("day", input), i); + + input = "1999.38888"; + i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); + assertEquals(fromSingleUnitString("second", input), i); + + try { + input = String.valueOf(Integer.MAX_VALUE); + fromSingleUnitString("year", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); } - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } + try { + input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); + fromSingleUnitString("hour", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); } + } - @Test - public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; + @Test + public void addTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; - interval = fromString(input); - interval2 = fromString(input2); + interval = fromString(input); + interval2 = fromString(input2); - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); - } + assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + } - @Test - public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; + @Test + public void subtractTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; - interval = fromString(input); - interval2 = fromString(input2); + interval = fromString(input); + interval2 = fromString(input2); - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } + assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + } - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - } -} \ No newline at end of file + private static void testSingleUnit(String unit, int number, int months, long microseconds) { + String input1 = "interval " + number + " " + unit; + String input2 = "interval " + number + " " + unit + "s"; + CalendarInterval result = new CalendarInterval(months, microseconds); + assertEquals(fromString(input1), result); + assertEquals(fromString(input2), result); + } +} From 44199dc60f35257d2afab25f34b538086d431e36 Mon Sep 17 00:00:00 2001 From: priyankagargnitk Date: Fri, 5 Jul 2019 15:28:13 +0530 Subject: [PATCH 03/13] Delete CalendarIntervalSuite.java --- .../sql/types/CalendarIntervalSuite.java | 268 ------------------ 1 file changed, 268 deletions(-) delete mode 100644 sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java deleted file mode 100644 index 317d074877c3..000000000000 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java +++ /dev/null @@ -1,268 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql.types; - -import org.junit.Test; - -import static org.junit.Assert.*; -import static org.apache.spark.sql.types.CalendarInterval.*; - -public class CalendarIntervalSuite { - - @Test - public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); - - assertNotSame(i1, i2); - assertNotSame(i1, i3); - assertNotSame(i2, i3); - assertEquals(i1, i4); - } - - @Test - 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()); - - i = new CalendarInterval(-34, 0); - assertEquals("interval -2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); - - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); - - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); - } - - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - String input; - - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); - - input = "interval -5 years 23 month "; - assertEquals(fromString(input), result); - - input = " interval -5 years 23 month "; - assertEquals(fromString(input), result); - - // Error cases - input = "interval 3month 1 hour"; - assertNull(fromString(input)); - - input = "interval 3 moth 1 hour"; - assertNull(fromString(input)); - - input = "interval"; - assertNull(fromString(input)); - - input = "int"; - assertNull(fromString(input)); - - input = ""; - assertNull(fromString(input)); - - input = null; - assertNull(fromString(input)); - } - - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("cannot be null or blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid interval")); - } - } - } - - @Test - public void fromYearMonthStringTest() { - String input; - CalendarInterval i; - - input = "99-10"; - i = new CalendarInterval(99 * 12 + 10, 0L); - assertEquals(fromYearMonthString(input), i); - - input = "-8-10"; - i = new CalendarInterval(-8 * 12 - 10, 0L); - assertEquals(fromYearMonthString(input), i); - - try { - input = "99-15"; - fromYearMonthString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("month 15 outside range")); - } - } - - @Test - public void fromDayTimeStringTest() { - String input; - CalendarInterval i; - - input = "5 12:40:30.999999999"; - i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); - assertEquals(fromDayTimeString(input), i); - - input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); - assertEquals(fromDayTimeString(input), i); - - input = "-3 0:0:0"; - i = new CalendarInterval(0, -3 * MICROS_PER_DAY); - assertEquals(fromDayTimeString(input), i); - - try { - input = "5 30:12:20"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("hour 30 outside range")); - } - - try { - input = "5 30-12"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("not match day-time format")); - } - } - - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - - @Test - public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); - } - - @Test - public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - } -} From d6decb295fc4f872f0896b4e1661a00446e3ab4a Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:31:07 +0530 Subject: [PATCH 04/13] SPARK-24695: To incorporate review comments --- .../unsafe/types/CalendarIntervalSuite.java | 268 ++++++++++++++++++ 1 file changed, 268 insertions(+) create mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java 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 new file mode 100644 index 000000000000..994af8f08244 --- /dev/null +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -0,0 +1,268 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.unsafe.types; + +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.apache.spark.unsafe.types.CalendarInterval.*; + +public class CalendarIntervalSuite { + + @Test + public void equalsTest() { + CalendarInterval i1 = new CalendarInterval(3, 123); + CalendarInterval i2 = new CalendarInterval(3, 321); + CalendarInterval i3 = new CalendarInterval(1, 123); + CalendarInterval i4 = new CalendarInterval(3, 123); + + assertNotSame(i1, i2); + assertNotSame(i1, i3); + assertNotSame(i2, i3); + assertEquals(i1, i4); + } + + @Test + 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()); + + i = new CalendarInterval(-34, 0); + assertEquals("interval -2 years -10 months", i.toString()); + + i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); + + i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); + assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + + i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); + } + + @Test + public void fromStringTest() { + testSingleUnit("year", 3, 36, 0); + testSingleUnit("month", 3, 3, 0); + testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); + testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); + testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); + testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); + testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); + testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); + testSingleUnit("microsecond", 3, 0, 3); + + String input; + + input = "interval -5 years 23 month"; + CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); + assertEquals(fromString(input), result); + + input = "interval -5 years 23 month "; + assertEquals(fromString(input), result); + + input = " interval -5 years 23 month "; + assertEquals(fromString(input), result); + + // Error cases + input = "interval 3month 1 hour"; + assertNull(fromString(input)); + + input = "interval 3 moth 1 hour"; + assertNull(fromString(input)); + + input = "interval"; + assertNull(fromString(input)); + + input = "int"; + assertNull(fromString(input)); + + input = ""; + assertNull(fromString(input)); + + input = null; + assertNull(fromString(input)); + } + + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); + } + + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } + } + + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + + @Test + public void fromYearMonthStringTest() { + String input; + CalendarInterval i; + + input = "99-10"; + i = new CalendarInterval(99 * 12 + 10, 0L); + assertEquals(fromYearMonthString(input), i); + + input = "-8-10"; + i = new CalendarInterval(-8 * 12 - 10, 0L); + assertEquals(fromYearMonthString(input), i); + + try { + input = "99-15"; + fromYearMonthString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("month 15 outside range")); + } + } + + @Test + public void fromDayTimeStringTest() { + String input; + CalendarInterval i; + + input = "5 12:40:30.999999999"; + i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); + assertEquals(fromDayTimeString(input), i); + + input = "10 0:12:0.888"; + i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); + assertEquals(fromDayTimeString(input), i); + + input = "-3 0:0:0"; + i = new CalendarInterval(0, -3 * MICROS_PER_DAY); + assertEquals(fromDayTimeString(input), i); + + try { + input = "5 30:12:20"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("hour 30 outside range")); + } + + try { + input = "5 30-12"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("not match day-time format")); + } + } + + @Test + public void fromSingleUnitStringTest() { + String input; + CalendarInterval i; + + input = "12"; + i = new CalendarInterval(12 * 12, 0L); + assertEquals(fromSingleUnitString("year", input), i); + + input = "100"; + i = new CalendarInterval(0, 100 * MICROS_PER_DAY); + assertEquals(fromSingleUnitString("day", input), i); + + input = "1999.38888"; + i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); + assertEquals(fromSingleUnitString("second", input), i); + + try { + input = String.valueOf(Integer.MAX_VALUE); + fromSingleUnitString("year", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + + try { + input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); + fromSingleUnitString("hour", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + } + + @Test + public void addTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + } + + @Test + public void subtractTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + } + + private static void testSingleUnit(String unit, int number, int months, long microseconds) { + String input1 = "interval " + number + " " + unit; + String input2 = "interval " + number + " " + unit + "s"; + CalendarInterval result = new CalendarInterval(months, microseconds); + assertEquals(fromString(input1), result); + assertEquals(fromString(input2), result); + } +} From 42b9ca4807eab2f004014363621fcd9c76120d2f Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:42:27 +0530 Subject: [PATCH 05/13] Revert "SPARK-24695: To incorporate review comments" This reverts commit d6decb295fc4f872f0896b4e1661a00446e3ab4a. --- .../unsafe/types/CalendarIntervalSuite.java | 268 ------------------ 1 file changed, 268 deletions(-) delete mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java 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 deleted file mode 100644 index 994af8f08244..000000000000 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ /dev/null @@ -1,268 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.unsafe.types; - -import org.junit.Test; - -import static org.junit.Assert.*; -import static org.apache.spark.unsafe.types.CalendarInterval.*; - -public class CalendarIntervalSuite { - - @Test - public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); - - assertNotSame(i1, i2); - assertNotSame(i1, i3); - assertNotSame(i2, i3); - assertEquals(i1, i4); - } - - @Test - 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()); - - i = new CalendarInterval(-34, 0); - assertEquals("interval -2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); - - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); - - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); - } - - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - String input; - - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); - - input = "interval -5 years 23 month "; - assertEquals(fromString(input), result); - - input = " interval -5 years 23 month "; - assertEquals(fromString(input), result); - - // Error cases - input = "interval 3month 1 hour"; - assertNull(fromString(input)); - - input = "interval 3 moth 1 hour"; - assertNull(fromString(input)); - - input = "interval"; - assertNull(fromString(input)); - - input = "int"; - assertNull(fromString(input)); - - input = ""; - assertNull(fromString(input)); - - input = null; - assertNull(fromString(input)); - } - - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("cannot be null or blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Invalid interval")); - } - } - } - - @Test - public void fromYearMonthStringTest() { - String input; - CalendarInterval i; - - input = "99-10"; - i = new CalendarInterval(99 * 12 + 10, 0L); - assertEquals(fromYearMonthString(input), i); - - input = "-8-10"; - i = new CalendarInterval(-8 * 12 - 10, 0L); - assertEquals(fromYearMonthString(input), i); - - try { - input = "99-15"; - fromYearMonthString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("month 15 outside range")); - } - } - - @Test - public void fromDayTimeStringTest() { - String input; - CalendarInterval i; - - input = "5 12:40:30.999999999"; - i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); - assertEquals(fromDayTimeString(input), i); - - input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); - assertEquals(fromDayTimeString(input), i); - - input = "-3 0:0:0"; - i = new CalendarInterval(0, -3 * MICROS_PER_DAY); - assertEquals(fromDayTimeString(input), i); - - try { - input = "5 30:12:20"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("hour 30 outside range")); - } - - try { - input = "5 30-12"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("not match day-time format")); - } - } - - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - - @Test - public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); - } - - @Test - public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - } -} From 6e69027dc49c9aa7fd4abf37360e3cf646b8cf58 Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:47:20 +0530 Subject: [PATCH 06/13] SPARK-24695: To incorporate review comments --- .../unsafe/types/CalendarIntervalSuite.java | 268 ++++++++++++++++++ 1 file changed, 268 insertions(+) create mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java 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 new file mode 100644 index 000000000000..994af8f08244 --- /dev/null +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -0,0 +1,268 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.unsafe.types; + +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.apache.spark.unsafe.types.CalendarInterval.*; + +public class CalendarIntervalSuite { + + @Test + public void equalsTest() { + CalendarInterval i1 = new CalendarInterval(3, 123); + CalendarInterval i2 = new CalendarInterval(3, 321); + CalendarInterval i3 = new CalendarInterval(1, 123); + CalendarInterval i4 = new CalendarInterval(3, 123); + + assertNotSame(i1, i2); + assertNotSame(i1, i3); + assertNotSame(i2, i3); + assertEquals(i1, i4); + } + + @Test + 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()); + + i = new CalendarInterval(-34, 0); + assertEquals("interval -2 years -10 months", i.toString()); + + i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); + + i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); + assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + + i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); + } + + @Test + public void fromStringTest() { + testSingleUnit("year", 3, 36, 0); + testSingleUnit("month", 3, 3, 0); + testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); + testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); + testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); + testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); + testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); + testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); + testSingleUnit("microsecond", 3, 0, 3); + + String input; + + input = "interval -5 years 23 month"; + CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); + assertEquals(fromString(input), result); + + input = "interval -5 years 23 month "; + assertEquals(fromString(input), result); + + input = " interval -5 years 23 month "; + assertEquals(fromString(input), result); + + // Error cases + input = "interval 3month 1 hour"; + assertNull(fromString(input)); + + input = "interval 3 moth 1 hour"; + assertNull(fromString(input)); + + input = "interval"; + assertNull(fromString(input)); + + input = "int"; + assertNull(fromString(input)); + + input = ""; + assertNull(fromString(input)); + + input = null; + assertNull(fromString(input)); + } + + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); + } + + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } + } + + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + + @Test + public void fromYearMonthStringTest() { + String input; + CalendarInterval i; + + input = "99-10"; + i = new CalendarInterval(99 * 12 + 10, 0L); + assertEquals(fromYearMonthString(input), i); + + input = "-8-10"; + i = new CalendarInterval(-8 * 12 - 10, 0L); + assertEquals(fromYearMonthString(input), i); + + try { + input = "99-15"; + fromYearMonthString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("month 15 outside range")); + } + } + + @Test + public void fromDayTimeStringTest() { + String input; + CalendarInterval i; + + input = "5 12:40:30.999999999"; + i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); + assertEquals(fromDayTimeString(input), i); + + input = "10 0:12:0.888"; + i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); + assertEquals(fromDayTimeString(input), i); + + input = "-3 0:0:0"; + i = new CalendarInterval(0, -3 * MICROS_PER_DAY); + assertEquals(fromDayTimeString(input), i); + + try { + input = "5 30:12:20"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("hour 30 outside range")); + } + + try { + input = "5 30-12"; + fromDayTimeString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("not match day-time format")); + } + } + + @Test + public void fromSingleUnitStringTest() { + String input; + CalendarInterval i; + + input = "12"; + i = new CalendarInterval(12 * 12, 0L); + assertEquals(fromSingleUnitString("year", input), i); + + input = "100"; + i = new CalendarInterval(0, 100 * MICROS_PER_DAY); + assertEquals(fromSingleUnitString("day", input), i); + + input = "1999.38888"; + i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); + assertEquals(fromSingleUnitString("second", input), i); + + try { + input = String.valueOf(Integer.MAX_VALUE); + fromSingleUnitString("year", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + + try { + input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); + fromSingleUnitString("hour", input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("outside range")); + } + } + + @Test + public void addTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + } + + @Test + public void subtractTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = fromString(input); + CalendarInterval interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = fromString(input); + interval2 = fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + } + + private static void testSingleUnit(String unit, int number, int months, long microseconds) { + String input1 = "interval " + number + " " + unit; + String input2 = "interval " + number + " " + unit + "s"; + CalendarInterval result = new CalendarInterval(months, microseconds); + assertEquals(fromString(input1), result); + assertEquals(fromString(input2), result); + } +} From 01f6e87e7639cb32e9fd4e3aab2445921a17adae Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:51:31 +0530 Subject: [PATCH 07/13] SPARK-24695: To incorporate review comments --- .../java/org/apache/spark/sql}/types/CalendarIntervalSuite.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename {common/unsafe/src/test/java/org/apache/spark/unsafe => sql/catalyst/src/test/java/org/apache/spark/sql}/types/CalendarIntervalSuite.java (100%) diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java similarity index 100% rename from common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java rename to sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java From 1d704eeb7e23969e1640f9cb2c9a095f8b2ad92b Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 15:52:10 +0530 Subject: [PATCH 08/13] SPARK-24695: To incorporate review comments --- .../org/apache/spark/sql/types/CalendarIntervalSuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java index 994af8f08244..317d074877c3 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.unsafe.types; +package org.apache.spark.sql.types; import org.junit.Test; import static org.junit.Assert.*; -import static org.apache.spark.unsafe.types.CalendarInterval.*; +import static org.apache.spark.sql.types.CalendarInterval.*; public class CalendarIntervalSuite { From 5330fcf8c81b065687f2513f497278395ca8c018 Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Fri, 5 Jul 2019 16:02:13 +0530 Subject: [PATCH 09/13] SPARK-24695: To incorporate documentation changes --- docs/sql-reference.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/sql-reference.md b/docs/sql-reference.md index 2ec26ecc2e1e..a1bc8c9e6e65 100644 --- a/docs/sql-reference.md +++ b/docs/sql-reference.md @@ -50,6 +50,9 @@ Spark SQL and DataFrames support the following data types: absolute point in time. - `DateType`: Represents values comprising values of fields year, month and day, without a time-zone. +* Calendar Interval type + - `CalendarIntervalType`: The data type representing calendar time intervals. The calendar time interval is stored + internally in two components: number of months and the number of microseconds. * Complex types - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of elements with the type of `elementType`. `containsNull` is used to indicate if @@ -163,6 +166,13 @@ You can access them by doing DateType + + CalendarIntervalType + org.apache.spark.sql.types.CalendarInterval + + CalendarIntervalType + + ArrayType scala.collection.Seq @@ -298,6 +308,13 @@ please use factory methods provided in DataTypes.DateType + + CalendarIntervalType + org.apache.spark.sql.types.CalendarInterval + + CalendarIntervalType + + ArrayType java.util.List From 97e73b0ab62a71508c50af85073d1c2edbc8fc5d Mon Sep 17 00:00:00 2001 From: prigarg Date: Tue, 16 Jul 2019 14:34:21 +0530 Subject: [PATCH 10/13] SPARK-24695: To add review comments --- docs/sql-reference.md | 9 +-------- .../org/apache/spark/sql/types/CalendarInterval.java | 9 +++++++-- .../src/main/scala/org/apache/spark/sql/Row.scala | 12 +++++++++++- .../spark/sql/catalyst/expressions/CastSuite.scala | 2 -- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 3 --- 5 files changed, 19 insertions(+), 16 deletions(-) diff --git a/docs/sql-reference.md b/docs/sql-reference.md index a1bc8c9e6e65..61a32a533249 100644 --- a/docs/sql-reference.md +++ b/docs/sql-reference.md @@ -50,7 +50,7 @@ Spark SQL and DataFrames support the following data types: absolute point in time. - `DateType`: Represents values comprising values of fields year, month and day, without a time-zone. -* Calendar Interval type +* CalendarInterval type - `CalendarIntervalType`: The data type representing calendar time intervals. The calendar time interval is stored internally in two components: number of months and the number of microseconds. * Complex types @@ -308,13 +308,6 @@ please use factory methods provided in DataTypes.DateType - - CalendarIntervalType - org.apache.spark.sql.types.CalendarInterval - - CalendarIntervalType - - ArrayType java.util.List diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java index 787efe70626f..143f03b1076c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java @@ -22,9 +22,13 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.spark.annotation.Unstable; + /** - * The internal representation of interval type. + * The representation class of interval type. */ + +@Unstable public final class CalendarInterval implements Serializable { public static final long MICROS_PER_MILLI = 1000L; public static final long MICROS_PER_SECOND = MICROS_PER_MILLI * 1000; @@ -103,7 +107,8 @@ public static CalendarInterval fromCaseInsensitiveString(String s) { throw new IllegalArgumentException("Interval cannot be null or blank."); } String sInLowerCase = s.trim().toLowerCase(Locale.ROOT); - String interval = sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; + String interval = + sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; CalendarInterval cal = fromString(interval); if (cal == null) { throw new IllegalArgumentException("Invalid interval: " + s); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 494387ecfe0c..2177b3da2ce7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -22,7 +22,7 @@ import scala.util.hashing.MurmurHash3 import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{CalendarInterval, StructType} /** * @since 1.3.0 @@ -159,6 +159,7 @@ trait Row extends Serializable { * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) * StructType -> org.apache.spark.sql.Row + * CalendarIntervalType -> org.apache.spark.sql.types.CalendarInterval * }}} */ def apply(i: Int): Any = get(i) @@ -184,6 +185,7 @@ trait Row extends Serializable { * ArrayType -> scala.collection.Seq (use getList for java.util.List) * MapType -> scala.collection.Map (use getJavaMap for java.util.Map) * StructType -> org.apache.spark.sql.Row + * CalendarIntervalType -> org.apache.spark.sql.types.CalendarInterval * }}} */ def get(i: Int): Any @@ -327,6 +329,14 @@ trait Row extends Serializable { */ def getStruct(i: Int): Row = getAs[Row](i) + /** + * Returns value at position i of calendar Interval type as a + * [[org.apache.spark.sql.types.CalendarInterval]] object. + * + * @throws ClassCastException when data type does not match. + */ + def getCalendarInterval(i: Int): CalendarInterval = CalendarInterval.fromString(get(i).toString) + /** * Returns the value at position i. * For primitive types if value is null it returns 'zero value' specific for primitive diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 228317dca935..795e9118ae98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -806,8 +806,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast between string and interval") { - import org.apache.spark.sql.types.CalendarInterval - checkEvaluation(Cast(Literal(""), CalendarIntervalType), null) checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType), new CalendarInterval(-3, 7 * CalendarInterval.MICROS_PER_HOUR)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a581ee2d9c96..2e358eaad0ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1469,8 +1469,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8753: add interval type") { - import org.apache.spark.sql.types.CalendarInterval - val df = sql("select interval 3 years -3 month 7 week 123 microseconds") checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * 1000 * 1000 * 3600 * 24 * 7 + 123 ))) withTempPath(f => { @@ -1494,7 +1492,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8945: add and subtract expressions for interval type") { - import org.apache.spark.sql.types.CalendarInterval import org.apache.spark.sql.types.CalendarInterval.MICROS_PER_WEEK val df = sql("select interval 3 years -3 month 7 week 123 microseconds as i") From 1b297e1b72574b1e9d5778e6ab3c723583634a2a Mon Sep 17 00:00:00 2001 From: prigarg Date: Tue, 16 Jul 2019 15:05:15 +0530 Subject: [PATCH 11/13] SPARK-24695: To incorporate review comments --- .../spark/sql/catalyst/encoders/RowEncoderSuite.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 5d21e4a2a83c..3b6f0e118b4d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -261,6 +261,16 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { assert(convertedBack.getSeq(2) == Seq(Seq(Seq(0L, null), null), null)) } + test("RowEncoder should support CalendarInterval as the external type for CalendarIntervalType") { + val schema = new StructType() + .add("calendarInterval", CalendarIntervalType) + val encoder = RowEncoder(schema).resolveAndBind() + val input = Row("interval '1' day") + val row = encoder.toRow(input) + val convertedBack = encoder.fromRow(row) + assert(convertedBack.getCalendarInterval(0) == CalendarInterval.fromString("interval '1' day")) + } + test("RowEncoder should throw RuntimeException if input row object is null") { val schema = new StructType().add("int", IntegerType) val encoder = RowEncoder(schema) From 0d33e22f4cd945c13842ee9419133e6156d92a26 Mon Sep 17 00:00:00 2001 From: prigarg Date: Wed, 24 Jul 2019 10:38:17 +0530 Subject: [PATCH 12/13] SPARK-24695: To add review comments --- .../org/apache/spark/sql/catalyst/expressions/literals.scala | 1 - .../sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala | 3 +-- .../spark/sql/execution/streaming/EventTimeWatermarkExec.scala | 3 +-- .../org/apache/spark/sql/execution/streaming/Triggers.scala | 2 +- 4 files changed, 3 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index d01140e6a836..8b642122efdb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -45,7 +45,6 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index ed6dcf729201..4247cc69270e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.CalendarInterval -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{CalendarInterval, IntegerType} class LeftSemiPushdownSuite extends PlanTest { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala index fe0057c6dbe3..09f51dd25cfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala @@ -23,8 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.types.CalendarInterval -import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.sql.types.{CalendarInterval, MetadataBuilder} import org.apache.spark.util.AccumulatorV2 /** Class for collecting event time stats with an accumulator */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index aede08820503..34473a96099d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -23,7 +23,7 @@ import scala.concurrent.duration.Duration import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval private object Triggers { def validate(intervalMs: Long): Unit = { From a07c79f430579295c17fc618d7722511652f3c4c Mon Sep 17 00:00:00 2001 From: prigarg Date: Wed, 24 Jul 2019 13:44:20 +0530 Subject: [PATCH 13/13] SPARK-24695: To fix failing test cases --- .../sql/catalyst/encoders/RowEncoder.scala | 8 ++++++++ .../sql/catalyst/ScalaReflectionSuite.scala | 10 ++++------ .../encoders/ExpressionEncoderSuite.scala | 17 ++++++++--------- 3 files changed, 20 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index afe8a23f8f15..97b41d4802b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -108,6 +108,14 @@ object RowEncoder { createSerializerForSqlDate(inputObject) } + case CalendarIntervalType => + StaticInvoke( + classOf[CalendarInterval], + CalendarIntervalType, + "fromString", + inputObject :: Nil, + returnNullable = false) + case d: DecimalType => CheckOverflow(StaticInvoke( Decimal.getClass, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 1412a8fc4da1..9111cf096b87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -34,8 +34,7 @@ case class PrimitiveData( floatField: Float, shortField: Short, byteField: Byte, - booleanField: Boolean, - intervalField: CalendarInterval) + booleanField: Boolean) case class NullableData( intField: java.lang.Integer, @@ -177,8 +176,7 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("shortField", ShortType, nullable = false), StructField("byteField", ByteType, nullable = false), StructField("booleanField", BooleanType, nullable = false))), - nullable = true), - StructField("intervalField", CalendarIntervalType, nullable = false)) + nullable = true)) } test("nullable data") { @@ -286,7 +284,7 @@ class ScalaReflectionSuite extends SparkFunSuite { } test("convert PrimitiveData to catalyst") { - val data = PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)) + val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true, new CalendarInterval(2, 0)) val dataType = schemaFor[PrimitiveData].dataType @@ -294,7 +292,7 @@ class ScalaReflectionSuite extends SparkFunSuite { } test("convert Option[Product] to catalyst") { - val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)) + val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), Some(primitiveData), Some(new CalendarInterval(2, 0))) val dataType = schemaFor[OptionalData].dataType diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 1ca7a7e8dedb..5f54c79b2877 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -203,11 +203,11 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Array(Option(InnerClass(1))), "array of optional inner class") - productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))) + productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true)) productTest( OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), - Some(PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))), + Some(PrimitiveData(1, 1, 1, 1, 1, 1, true)), Some(new CalendarInterval(2, 0)))) productTest(OptionalData(None, None, None, None, None, None, None, None, None)) @@ -219,10 +219,9 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest(BoxedData(null, null, null, null, null, null, null)) - productTest(RepeatedStruct(PrimitiveData(1, 1, 1, 1, 1, 1, true, - new CalendarInterval(2, 0)) :: Nil)) + productTest(RepeatedStruct(PrimitiveData(1, 1, 1, 1, 1, 1, true) :: Nil)) - productTest((1, "test", PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)))) + productTest((1, "test", PrimitiveData(1, 1, 1, 1, 1, 1, true))) productTest( RepeatedData( @@ -230,7 +229,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes Seq(Integer.valueOf(1), null, Integer.valueOf(2)), Map(1 -> 2L), Map(1 -> null), - PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)))) + PrimitiveData(1, 1, 1, 1, 1, 1, true))) productTest(NestedArray(Array(Array(1, -2, 3), null, Array(4, 5, -6)))) @@ -278,17 +277,17 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[Long])) encodeDecodeTest( - (PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)), (3, 30L)), + (PrimitiveData(1, 1, 1, 1, 1, 1, true), (3, 30L)), "tuple with 2 product encoders")( ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[(Int, Long)])) encodeDecodeTest( - (PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0)), 3), + (PrimitiveData(1, 1, 1, 1, 1, 1, true), 3), "tuple with flat encoder and product encoder")( ExpressionEncoder.tuple(ExpressionEncoder[PrimitiveData], ExpressionEncoder[Int])) encodeDecodeTest( - (3, PrimitiveData(1, 1, 1, 1, 1, 1, true, new CalendarInterval(2, 0))), + (3, PrimitiveData(1, 1, 1, 1, 1, 1, true)), "tuple with product encoder and flat encoder")( ExpressionEncoder.tuple(ExpressionEncoder[Int], ExpressionEncoder[PrimitiveData]))