From bd805299cc9802597c165a6de1667a7b02ad48ae Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Sat, 30 Jun 2018 12:53:56 +0530 Subject: [PATCH 1/5] 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 --- .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../sql/catalyst/ScalaReflectionSuite.scala | 35 ++++++++++++------- .../encoders/ExpressionEncoderSuite.scala | 22 ++++++------ 3 files changed, 35 insertions(+), 23 deletions(-) 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 f9acc208b715..7472f1f2b68e 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 @@ -751,6 +751,7 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) + case t if t <:< localTypeOf[CalendarInterval] => Schema(CalendarIntervalType, nullable = true) case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if t <:< localTypeOf[java.math.BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) 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 353b8344658f..9353cc8669da 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, Literal, SpecificInternalRow, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, NewInstance} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} case class PrimitiveData( intField: Int, @@ -33,7 +33,8 @@ case class PrimitiveData( floatField: Float, shortField: Short, byteField: Byte, - booleanField: Boolean) + booleanField: Boolean, + intervalField: CalendarInterval) case class NullableData( intField: java.lang.Integer, @@ -47,7 +48,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], @@ -57,7 +59,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], @@ -131,7 +134,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") { @@ -149,7 +153,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)) } @@ -164,7 +169,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)) } @@ -205,7 +212,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", @@ -235,19 +243,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/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index e6d09bdae67d..5b087410a95f 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.ClosureCleaner case class RepeatedStruct(s: Seq[PrimitiveData]) @@ -202,13 +202,14 @@ class ExpressionEncoderSuite extends PlanTest with AnalysisTest { 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 PlanTest with AnalysisTest { 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 PlanTest with AnalysisTest { Seq(new Integer(1), null, new Integer(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 PlanTest with AnalysisTest { 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])) From 357cb258c4cc3373569801936671ce03ea1fe708 Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Thu, 20 Jun 2019 23:12:27 +0530 Subject: [PATCH 2/5] SPARK-24695: to move Calendar Interval class from unsafe to public classes --- .../spark/unsafe/types/CalendarInterval.java | 332 ------------- .../unsafe/types/CalendarIntervalSuite.java | 240 --------- .../expressions/SpecializedGetters.java | 4 +- .../catalyst/expressions/UnsafeArrayData.java | 1 - .../sql/catalyst/expressions/UnsafeRow.java | 1 - .../expressions/codegen/UnsafeWriter.java | 6 +- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../analysis/StreamingJoinHelper.scala | 2 - .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/JoinedRow.scala | 2 +- .../sql/catalyst/expressions/arithmetic.scala | 3 +- .../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 | 2 +- .../expressions/objects/objects.scala | 2 +- .../spark/sql/catalyst/expressions/rows.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 1 - .../plans/logical/EventTimeWatermark.scala | 2 +- .../sql/catalyst/util/GenericArrayData.scala | 4 +- .../spark/sql/types/CalendarInterval.scala | 286 +++++++++++ .../spark/sql/RandomDataGenerator.scala | 1 - .../sql/catalyst/ScalaReflectionSuite.scala | 2 +- .../catalyst/analysis/TypeCoercionSuite.scala | 1 - .../analysis/UnsupportedOperationsSuite.scala | 3 +- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 2 +- .../CollectionExpressionsSuite.scala | 13 +- .../expressions/DateExpressionsSuite.scala | 1 - .../expressions/HashExpressionsSuite.scala | 2 +- .../expressions/LiteralExpressionSuite.scala | 2 - .../expressions/LiteralGenerator.scala | 1 - .../expressions/ObjectExpressionsSuite.scala | 2 +- .../GenerateUnsafeProjectionSuite.scala | 4 +- .../optimizer/FilterPushdownSuite.scala | 2 +- .../sql/types/CalendarIntervalSuite.scala | 454 ++++++++++++++++++ .../vectorized/ColumnVectorUtils.java | 10 +- .../vectorized/MutableColumnarRow.java | 1 - .../spark/sql/vectorized/ColumnVector.java | 2 +- .../spark/sql/vectorized/ColumnarArray.java | 1 - .../spark/sql/vectorized/ColumnarRow.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 +- .../sql/DataFrameWindowFramesSuite.scala | 2 +- .../apache/spark/sql/DateFunctionsSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../vectorized/ColumnarBatchSuite.scala | 1 - .../catalyst/ExpressionSQLBuilderSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 1 - 55 files changed, 794 insertions(+), 648 deletions(-) delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java delete mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java deleted file mode 100644 index 621f2c6bf377..000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ /dev/null @@ -1,332 +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 java.io.Serializable; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * The internal representation of interval type. - */ -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; - public static final long MICROS_PER_MINUTE = MICROS_PER_SECOND * 60; - public static final long MICROS_PER_HOUR = MICROS_PER_MINUTE * 60; - public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; - public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; - - /** - * A function to generate regex which matches interval string's unit part like "3 years". - * - * First, we can leave out some units in interval string, and we only care about the value of - * unit, so here we use non-capturing group to wrap the actual regex. - * At the beginning of the actual regex, we should match spaces before the unit part. - * Next is the number part, starts with an optional "-" to represent negative value. We use - * capturing group to wrap this part as we need the value later. - * Finally is the unit name, ends with an optional "s". - */ - private static String unitRegex(String unit) { - return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?"; - } - - private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")); - - private static Pattern yearMonthPattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); - - private static Pattern dayTimePattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+) (\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); - - private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); - - private static long toLong(String s) { - if (s == null) { - return 0; - } else { - return Long.parseLong(s); - } - } - - public static CalendarInterval fromString(String s) { - if (s == null) { - return null; - } - s = s.trim(); - Matcher m = p.matcher(s); - if (!m.matches() || s.equals("interval")) { - return null; - } else { - long months = toLong(m.group(1)) * 12 + toLong(m.group(2)); - long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK; - microseconds += toLong(m.group(4)) * MICROS_PER_DAY; - microseconds += toLong(m.group(5)) * MICROS_PER_HOUR; - microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE; - microseconds += toLong(m.group(7)) * MICROS_PER_SECOND; - microseconds += toLong(m.group(8)) * MICROS_PER_MILLI; - microseconds += toLong(m.group(9)); - return new CalendarInterval((int) months, microseconds); - } - } - - public static long toLongWithRange(String fieldName, - String s, long minValue, long maxValue) throws IllegalArgumentException { - long result = 0; - if (s != null) { - result = Long.parseLong(s); - if (result < minValue || result > maxValue) { - throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]", - fieldName, result, minValue, maxValue)); - } - } - return result; - } - - /** - * Parse YearMonth string in form: [-]YYYY-MM - * - * adapted from HiveIntervalYearMonth.valueOf - */ - public static CalendarInterval fromYearMonthString(String s) throws IllegalArgumentException { - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException("Interval year-month string was null"); - } - s = s.trim(); - Matcher m = yearMonthPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match year-month format of 'y-m': " + s); - } else { - try { - int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; - int years = (int) toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE); - int months = (int) toLongWithRange("month", m.group(3), 0, 11); - result = new CalendarInterval(sign * (years * 12 + months), 0); - } catch (Exception e) { - throw new IllegalArgumentException( - "Error parsing interval year-month string: " + e.getMessage(), e); - } - } - return result; - } - - /** - * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn - * - * adapted from HiveIntervalDayTime.valueOf - */ - public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException { - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException("Interval day-time string was null"); - } - s = s.trim(); - Matcher m = dayTimePattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { - try { - int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; - long days = toLongWithRange("day", m.group(2), 0, Integer.MAX_VALUE); - long hours = toLongWithRange("hour", m.group(3), 0, 23); - long minutes = toLongWithRange("minute", m.group(4), 0, 59); - long seconds = toLongWithRange("second", m.group(5), 0, 59); - // Hive allow nanosecond precision interval - long nanos = toLongWithRange("nanosecond", m.group(7), 0L, 999999999L); - result = new CalendarInterval(0, sign * ( - days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + - seconds * MICROS_PER_SECOND + nanos / 1000L)); - } catch (Exception e) { - throw new IllegalArgumentException( - "Error parsing interval day-time string: " + e.getMessage(), e); - } - } - return result; - } - - public static CalendarInterval fromSingleUnitString(String unit, String s) - throws IllegalArgumentException { - - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException(String.format("Interval %s string was null", unit)); - } - s = s.trim(); - Matcher m = quoteTrimPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { - try { - switch (unit) { - case "year": - int year = (int) toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); - result = new CalendarInterval(year * 12, 0L); - break; - case "month": - int month = (int) toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE); - result = new CalendarInterval(month, 0L); - break; - case "week": - long week = toLongWithRange("week", m.group(1), - Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); - result = new CalendarInterval(0, week * MICROS_PER_WEEK); - break; - case "day": - long day = toLongWithRange("day", m.group(1), - Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); - result = new CalendarInterval(0, day * MICROS_PER_DAY); - break; - case "hour": - long hour = toLongWithRange("hour", m.group(1), - Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); - result = new CalendarInterval(0, hour * MICROS_PER_HOUR); - break; - case "minute": - long minute = toLongWithRange("minute", m.group(1), - Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); - result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); - break; - case "second": { - long micros = parseSecondNano(m.group(1)); - result = new CalendarInterval(0, micros); - break; - } - case "millisecond": - long millisecond = toLongWithRange("millisecond", m.group(1), - Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); - result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); - break; - case "microsecond": { - long micros = Long.parseLong(m.group(1)); - result = new CalendarInterval(0, micros); - break; - } - } - } catch (Exception e) { - throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); - } - } - return result; - } - - /** - * Parse second_nano string in ss.nnnnnnnnn format to microseconds - */ - public static long parseSecondNano(String secondNano) throws IllegalArgumentException { - String[] parts = secondNano.split("\\."); - if (parts.length == 1) { - return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND, - Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND; - - } else if (parts.length == 2) { - long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0], - Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND); - long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L); - return seconds * MICROS_PER_SECOND + nanos / 1000L; - - } else { - throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn"); - } - } - - public final int months; - public final long microseconds; - - public long milliseconds() { - return this.microseconds / MICROS_PER_MILLI; - } - - public CalendarInterval(int months, long microseconds) { - this.months = months; - this.microseconds = microseconds; - } - - public CalendarInterval add(CalendarInterval that) { - int months = this.months + that.months; - long microseconds = this.microseconds + that.microseconds; - return new CalendarInterval(months, microseconds); - } - - public CalendarInterval subtract(CalendarInterval that) { - int months = this.months - that.months; - long microseconds = this.microseconds - that.microseconds; - return new CalendarInterval(months, microseconds); - } - - public CalendarInterval negate() { - return new CalendarInterval(-this.months, -this.microseconds); - } - - @Override - public boolean equals(Object other) { - if (this == other) return true; - if (other == null || !(other instanceof CalendarInterval)) return false; - - CalendarInterval o = (CalendarInterval) other; - return this.months == o.months && this.microseconds == o.microseconds; - } - - @Override - public int hashCode() { - return 31 * months + (int) microseconds; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("interval"); - - if (months != 0) { - appendUnit(sb, months / 12, "year"); - appendUnit(sb, months % 12, "month"); - } - - if (microseconds != 0) { - long rest = microseconds; - appendUnit(sb, rest / MICROS_PER_WEEK, "week"); - rest %= MICROS_PER_WEEK; - appendUnit(sb, rest / MICROS_PER_DAY, "day"); - rest %= MICROS_PER_DAY; - appendUnit(sb, rest / MICROS_PER_HOUR, "hour"); - rest %= MICROS_PER_HOUR; - appendUnit(sb, rest / MICROS_PER_MINUTE, "minute"); - rest %= MICROS_PER_MINUTE; - appendUnit(sb, rest / MICROS_PER_SECOND, "second"); - rest %= MICROS_PER_SECOND; - appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond"); - rest %= MICROS_PER_MILLI; - appendUnit(sb, rest, "microsecond"); - } - - return sb.toString(); - } - - private void appendUnit(StringBuilder sb, long value, String unit) { - if (value != 0) { - sb.append(' ').append(value).append(' ').append(unit).append('s'); - } - } -} 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 9e69e264ff28..000000000000 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ /dev/null @@ -1,240 +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(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 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 4dd2b7365652..13146067a303 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 @@ -28,7 +28,6 @@ import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; import org.apache.spark.unsafe.memory.MemoryBlock; -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/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 469b0e60cc9a..d9d5da77e82a 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 @@ -38,7 +38,6 @@ import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; import org.apache.spark.unsafe.memory.MemoryBlock; -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 278165500200..02d83aa5bb91 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; /** @@ -135,8 +135,8 @@ public final void write(int ordinal, CalendarInterval input) { grow(16); // Write the months and microseconds fields of Interval to the variable length portion. - Platform.putLong(getBuffer(), cursor(), input.months); - Platform.putLong(getBuffer(), cursor() + 8, input.microseconds); + Platform.putLong(getBuffer(), cursor(), input.months()); + Platform.putLong(getBuffer(), cursor() + 8, input.microseconds()); setOffsetAndSize(ordinal, 16); 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 7472f1f2b68e..8e39ae2c4f89 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** 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 7a0aa08289ef..aa7b849a5292 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 699ea53b5df0..e521f4288848 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ 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.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/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index fe91e520169b..039d2511c04c 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 @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval @ExpressionDescription( usage = "_FUNC_(expr) - Returns the negated value of `expr`.", @@ -58,7 +57,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression protected override def nullSafeEval(input: Any): Any = { if (dataType.isInstanceOf[CalendarIntervalType]) { - input.asInstanceOf[CalendarInterval].negate() + input.asInstanceOf[CalendarInterval].negate } else { numeric.negate(input) } 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 4cc0968911cb..f575f1d20b07 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 @@ -44,7 +44,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.array.ByteArrayMethods -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 8b278f067749..9af03aeeeee2 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.Platform 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 08838d2b2c61..2295fe05fc8a 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.DateTimeUtils 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 cec00b66f873..842a34c7efa2 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 @@ -34,7 +34,7 @@ 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.memory.MemoryBlock -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 0cc2a332f2c3..b8a0d27910bc 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 @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, Scala import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types._ +import org.apache.spark.unsafe.types.UTF8String object Literal { val TrueLiteral: Literal = Literal(true, BooleanType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 2bf4203d0fec..b642bbfe7ae3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} 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.Utils /** 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 383ebde3229d..883bcf5822b1 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 @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval 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 7a927e1e083b..3d91afb25f31 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 @@ -18,8 +18,8 @@ package org.apache.spark.sql.catalyst.plans.logical 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/main/scala/org/apache/spark/sql/types/CalendarInterval.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala new file mode 100644 index 000000000000..d8c19f22fc6f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala @@ -0,0 +1,286 @@ +/* + * 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 java.io.Serializable +import java.util.regex.Pattern + + +/** + * The public representation of interval type. + */ + +object CalendarInterval { + val MICROS_PER_MILLI = 1000L + val MICROS_PER_SECOND: Long = MICROS_PER_MILLI * 1000 + val MICROS_PER_MINUTE: Long = MICROS_PER_SECOND * 60 + val MICROS_PER_HOUR: Long = MICROS_PER_MINUTE * 60 + val MICROS_PER_DAY: Long = MICROS_PER_HOUR * 24 + val MICROS_PER_WEEK: Long = MICROS_PER_DAY * 7 + +/** + * A function to generate regex which matches interval string's unit part like "3 years". + * + * First, we can leave out some units in interval string, and we only care about the value of + * unit, so here we use non-capturing group to wrap the actual regex. + * At the beginning of the actual regex, we should match spaces before the unit part. + * Next is the number part, starts with an optional "-" to represent negative value. We use + * capturing group to wrap this part as we need the value later. + * Finally is the unit name, ends with an optional "s". + */ + private def unitRegex(unit: String) = "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?" + + private val p = Pattern.compile("interval" + unitRegex("year") + + unitRegex("month") + unitRegex("week") + unitRegex("day") + + unitRegex("hour") + unitRegex("minute") + unitRegex("second") + + unitRegex("millisecond") + unitRegex("microsecond")) + private val yearMonthPattern = Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$") + private val dayTimePattern = Pattern.compile( + "^(?:['|\"])?([+|-])?(\\d+) (\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$") + private val quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$") + + private def toLong(s: String) = if (s == null) 0L else s.toLong + + def fromString(s: String): CalendarInterval = { + if (s == null) return null + val m = p.matcher(s.trim) + if (!m.matches || s.trim == "interval") null + else { + val months = toLong(m.group(1)) * 12 + toLong(m.group(2)) + var microseconds = toLong(m.group(3)) * MICROS_PER_WEEK + microseconds += toLong(m.group(4)) * MICROS_PER_DAY + microseconds += toLong(m.group(5)) * MICROS_PER_HOUR + microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE + microseconds += toLong(m.group(7)) * MICROS_PER_SECOND + microseconds += toLong(m.group(8)) * MICROS_PER_MILLI + microseconds += toLong(m.group(9)) + new CalendarInterval(months.toInt, microseconds) + } + } + + @throws[IllegalArgumentException] + def toLongWithRange(fieldName: String, s: String, minValue: Long, maxValue: Long): Long = { + var result = 0L + if (s != null) { + result = s.toLong + if (result < minValue || result > maxValue) { + throw new IllegalArgumentException( + s"${fieldName} ${result} outside range [${minValue}, ${maxValue}") + } + } + result + } + +/** + * Parse YearMonth string in form: [-]YYYY-MM + * + * adapted from HiveIntervalYearMonth.valueOf + */ + @throws[IllegalArgumentException] + def fromYearMonthString(s: String): CalendarInterval = { + var result : CalendarInterval = null + if (s == null) throw new IllegalArgumentException("Interval year-month string was null") + val m = yearMonthPattern.matcher(s.trim) + if (!m.matches) { + throw new IllegalArgumentException( + "Interval string does not match year-month format of 'y-m': " + s) + } else try { + val sign = if (m.group(1) != null && m.group(1) == "-") -1 + else 1 + val years = toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE).toInt + val months = toLongWithRange("month", m.group(3), 0, 11).toInt + result = new CalendarInterval(sign * (years * 12 + months), 0) + } catch { + case e: Exception => + throw new IllegalArgumentException( + "Error parsing interval year-month string: " + e.getMessage, e) + } + result + } + +/** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf + */ + @throws[IllegalArgumentException] + def fromDayTimeString(s: String): CalendarInterval = { + var result : CalendarInterval = null + if (s == null) throw new IllegalArgumentException("Interval day-time string was null") + val m = dayTimePattern.matcher(s.trim) + if (!m.matches) { + throw new IllegalArgumentException( + "Interval string does not match day-time format of 'd h:m:s.n': " + s) + } + else try { + val sign = if (m.group(1) != null && m.group(1) == "-") -1 + else 1 + val days = toLongWithRange("day", m.group(2), 0, Integer.MAX_VALUE) + val hours = toLongWithRange("hour", m.group(3), 0, 23) + val minutes = toLongWithRange("minute", m.group(4), 0, 59) + val seconds = toLongWithRange("second", m.group(5), 0, 59) + // Hive allow nanosecond precision interval + val nanos = toLongWithRange("nanosecond", m.group(7), 0L, 999999999L) + result = new CalendarInterval(0, sign * (days * MICROS_PER_DAY + + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + + seconds * MICROS_PER_SECOND + nanos / 1000L)) + } catch { + case e: Exception => + throw new IllegalArgumentException( + "Error parsing interval day-time string: " + e.getMessage, e) + } + result + } + + @throws[IllegalArgumentException] + def fromSingleUnitString(unit: String, s: String): CalendarInterval = { + var result : CalendarInterval = null + if (s == null) { + throw new IllegalArgumentException(String.format("Interval %s string was null", unit)) + } + + val m = quoteTrimPattern.matcher(s.trim) + if (!m.matches) { + throw new IllegalArgumentException( + "Interval string does not match day-time format of 'd h:m:s.n': " + s) + } + else try + unit match { + case "year" => + val year = toLongWithRange("year", m.group(1), + Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12).toInt + result = new CalendarInterval(year * 12, 0L) + case "month" => + val month = toLongWithRange("month", m.group(1), + Integer.MIN_VALUE, Integer.MAX_VALUE).toInt + result = new CalendarInterval(month, 0L) + case "week" => + val week = toLongWithRange("week", m.group(1), + Long.MinValue / MICROS_PER_WEEK, Long.MaxValue / MICROS_PER_WEEK) + result = new CalendarInterval(0, week * MICROS_PER_WEEK) + case "day" => + val day = toLongWithRange("day", m.group(1), + Long.MinValue / MICROS_PER_DAY, Long.MaxValue / MICROS_PER_DAY) + result = new CalendarInterval(0, day * MICROS_PER_DAY) + case "hour" => + val hour = toLongWithRange("hour", m.group(1), + Long.MinValue / MICROS_PER_HOUR, Long.MaxValue / MICROS_PER_HOUR) + result = new CalendarInterval(0, hour * MICROS_PER_HOUR) + case "minute" => + val minute = toLongWithRange("minute", m.group(1), + Long.MinValue / MICROS_PER_MINUTE, Long.MaxValue / MICROS_PER_MINUTE) + result = new CalendarInterval(0, minute * MICROS_PER_MINUTE) + case "second" => + val micros = parseSecondNano(m.group(1)) + result = new CalendarInterval(0, micros) + case "millisecond" => + val millisecond = toLongWithRange("millisecond", m.group(1), + Long.MinValue / MICROS_PER_MILLI, Long.MaxValue / MICROS_PER_MILLI) + result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI) + case "microsecond" => + val micros = m.group(1).toLong + result = new CalendarInterval(0, micros) + } + catch { + case e: Exception => + throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage, e) + } + result + } + +/** + * Parse second_nano string in ss.nnnnnnnnn format to microseconds + */ + @throws[IllegalArgumentException] + def parseSecondNano(secondNano: String): Long = { + val parts = secondNano.split("\\.") + if (parts.length == 1) { + toLongWithRange("second", parts(0), Long.MinValue / MICROS_PER_SECOND, Long.MaxValue / + MICROS_PER_SECOND) * MICROS_PER_SECOND + } + else if (parts.length == 2) { + val seconds = if (parts(0) == "") 0L + else { + toLongWithRange("second", parts(0), + Long.MinValue / MICROS_PER_SECOND, Long.MaxValue / MICROS_PER_SECOND) + } + val nanos = toLongWithRange("nanosecond", parts(1), 0L, 999999999L) + seconds * MICROS_PER_SECOND + nanos / 1000L + } + else { + throw new IllegalArgumentException( + "Interval string does not match second-nano format of ss.nnnnnnnnn") + } + } +} + +final case class CalendarInterval(val months: Int, val microseconds: Long) extends Serializable { + def milliseconds: Long = this.microseconds / CalendarInterval.MICROS_PER_MILLI + + def add(that: CalendarInterval): CalendarInterval = { + val months = this.months + that.months + val microseconds = this.microseconds + that.microseconds + new CalendarInterval(months, microseconds) + } + + def subtract(that: CalendarInterval): CalendarInterval = { + val months = this.months - that.months + val microseconds = this.microseconds - that.microseconds + new CalendarInterval(months, microseconds) + } + + def negate : CalendarInterval = new CalendarInterval(-this.months, -this.microseconds) + + override def equals(other: Any): Boolean = { + if (this equals other) return true + if (other == null || !other.isInstanceOf[CalendarInterval]) return false + val o = other.asInstanceOf[CalendarInterval] + this.months == o.months && this.microseconds == o.microseconds + } + + override def hashCode: Int = 31 * months + microseconds.toInt + + override def toString: String = { + val sb = new StringBuilder("interval") + if (months != 0) { + appendUnit(sb, months / 12, "year") + appendUnit(sb, months % 12, "month") + } + if (microseconds != 0) { + var rest = microseconds + appendUnit(sb, rest / CalendarInterval.MICROS_PER_WEEK, "week") + rest %= CalendarInterval.MICROS_PER_WEEK + appendUnit(sb, rest / CalendarInterval.MICROS_PER_DAY, "day") + rest %= CalendarInterval.MICROS_PER_DAY + appendUnit(sb, rest / CalendarInterval.MICROS_PER_HOUR, "hour") + rest %= CalendarInterval.MICROS_PER_HOUR + appendUnit(sb, rest / CalendarInterval.MICROS_PER_MINUTE, "minute") + rest %= CalendarInterval.MICROS_PER_MINUTE + appendUnit(sb, rest / CalendarInterval.MICROS_PER_SECOND, "second") + rest %= CalendarInterval.MICROS_PER_SECOND + appendUnit(sb, rest / CalendarInterval.MICROS_PER_MILLI, "millisecond") + rest %= CalendarInterval.MICROS_PER_MILLI + appendUnit(sb, rest, "microsecond") + } + sb.toString + } + + private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { + if (value != 0) sb.append(' ').append(value).append(' ').append(unit).append('s') + } +} 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 8ae3ff5043e6..91e93ddfb0a2 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 @@ -27,7 +27,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 9353cc8669da..7ef1bb5b5d5f 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, Literal, SpecificInternalRow, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, NewInstance} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String case class PrimitiveData( intField: Int, 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 0acd3b490447..871fd6d32ee2 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 cb487c889354..14ab60131bae 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 5b087410a95f..b009ccecd930 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ArrayData 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.ClosureCleaner case class RepeatedStruct(s: Seq[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 5b25bdf907c3..ff9b887bb46f 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 @@ -773,7 +773,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 d7744eb4c7dc..aca31026500e 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeTestUtils 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 { @@ -556,7 +555,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(CalendarInterval.fromString("interval 12 hours").negate)), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -565,7 +564,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(CalendarInterval.fromString("interval 12 hours").negate)), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -583,7 +582,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(CalendarInterval.fromString("interval 1 month").negate)), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -592,7 +591,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 day").negate())), + Literal(CalendarInterval.fromString("interval 1 month 1 day").negate)), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -638,7 +637,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2022-04-01 00:00:00")), Literal(Timestamp.valueOf("2017-01-01 00:00:00")), - Literal(CalendarInterval.fromYearMonthString("1-5").negate())), + Literal(CalendarInterval.fromYearMonthString("1-5").negate)), Seq( Timestamp.valueOf("2022-04-01 00:00:00.000"), Timestamp.valueOf("2020-11-01 00:00:00.000"), @@ -730,7 +729,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(CalendarInterval.fromString("interval 1 month").negate)), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * CalendarInterval.MICROS_PER_DAY}") } 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 63b24fb9eb13..2d5985ae63c2 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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval 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 4281c89ac475..07bbe906acb6 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 86f80fe66d28..1df273a161ce 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 @@ -27,8 +27,6 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.util.DateTimeUtils 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 032aec01782f..67b203352582 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 @@ -22,7 +22,6 @@ import java.sql.{Date, Timestamp} import org.scalacheck.{Arbitrary, Gen} 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/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index 20d568c44258..5236450d27a4 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 @@ -38,7 +38,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/codegen/GenerateUnsafeProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala index e9d21f8a8ebc..1b33406b3a1a 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 @@ -21,8 +21,8 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.BoundReference import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} -import org.apache.spark.sql.types.{DataType, Decimal, StringType, StructType} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.sql.types.{CalendarInterval, DataType, Decimal, StringType, StructType} +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 82a10254d846..9b8f9ceabf28 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 @@ -25,8 +25,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/types/CalendarIntervalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala new file mode 100644 index 000000000000..1cf0cd0a93e5 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala @@ -0,0 +1,454 @@ +/* + * 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 com.fasterxml.jackson.core.JsonParseException +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.{SparkException, SparkFunSuite} + +class CalendarIntervalSuite extends SparkFunSuite { + + test("construct an ArrayType") { + val array = ArrayType(StringType) + + assert(ArrayType(StringType, true) === array) + } + + test("construct an MapType") { + val map = MapType(StringType, IntegerType) + + assert(MapType(StringType, IntegerType, true) === map) + } + + test("construct with add") { + val struct = (new StructType) + .add("a", IntegerType, true) + .add("b", LongType, false) + .add("c", StringType, true) + + assert(StructField("b", LongType, false) === struct("b")) + } + + test("construct with add from StructField") { + // Test creation from StructField type + val struct = (new StructType) + .add(StructField("a", IntegerType, true)) + .add(StructField("b", LongType, false)) + .add(StructField("c", StringType, true)) + + assert(StructField("b", LongType, false) === struct("b")) + } + + test("construct with add from StructField with comments") { + // Test creation from StructField using four different ways + val struct = (new StructType) + .add("a", "int", true, "test1") + .add("b", StringType, true, "test3") + .add(StructField("c", LongType, false).withComment("test4")) + .add(StructField("d", LongType)) + + assert(StructField("a", IntegerType, true).withComment("test1") == struct("a")) + assert(StructField("b", StringType, true).withComment("test3") == struct("b")) + assert(StructField("c", LongType, false).withComment("test4") == struct("c")) + assert(StructField("d", LongType) == struct("d")) + + assert(struct("c").getComment() == Option("test4")) + assert(struct("d").getComment().isEmpty) + } + + test("construct with String DataType") { + // Test creation with DataType as String + val struct = (new StructType) + .add("a", "int", true) + .add("b", "long", false) + .add("c", "string", true) + + assert(StructField("a", IntegerType, true) === struct("a")) + assert(StructField("b", LongType, false) === struct("b")) + assert(StructField("c", StringType, true) === struct("c")) + } + + test("extract fields from a StructType") { + val struct = StructType( + StructField("a", IntegerType, true) :: + StructField("b", LongType, false) :: + StructField("c", StringType, true) :: + StructField("d", FloatType, true) :: Nil) + + assert(StructField("b", LongType, false) === struct("b")) + + intercept[IllegalArgumentException] { + struct("e") + } + + val expectedStruct = StructType( + StructField("b", LongType, false) :: + StructField("d", FloatType, true) :: Nil) + + assert(expectedStruct === struct(Set("b", "d"))) + intercept[IllegalArgumentException] { + struct(Set("b", "d", "e", "f")) + } + } + + test("extract field index from a StructType") { + val struct = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + + assert(struct.fieldIndex("a") === 0) + assert(struct.fieldIndex("b") === 1) + + intercept[IllegalArgumentException] { + struct.fieldIndex("non_existent") + } + } + + test("fieldsMap returns map of name to StructField") { + val struct = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + + val mapped = StructType.fieldsMap(struct.fields) + + val expected = Map( + "a" -> StructField("a", LongType), + "b" -> StructField("b", FloatType)) + + assert(mapped === expected) + } + + test("fieldNames and names returns field names") { + val struct = StructType( + StructField("a", LongType) :: StructField("b", FloatType) :: Nil) + + assert(struct.fieldNames === Seq("a", "b")) + assert(struct.names === Seq("a", "b")) + } + + test("merge where right contains type conflict") { + val left = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + + val right = StructType( + StructField("b", LongType) :: Nil) + + val message = intercept[SparkException] { + left.merge(right) + }.getMessage + assert(message.equals("Failed to merge fields 'b' and 'b'. " + + "Failed to merge incompatible data types FloatType and LongType")) + } + + test("existsRecursively") { + val struct = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + assert(struct.existsRecursively(_.isInstanceOf[LongType])) + assert(struct.existsRecursively(_.isInstanceOf[StructType])) + assert(!struct.existsRecursively(_.isInstanceOf[IntegerType])) + + val mapType = MapType(struct, StringType) + assert(mapType.existsRecursively(_.isInstanceOf[LongType])) + assert(mapType.existsRecursively(_.isInstanceOf[StructType])) + assert(mapType.existsRecursively(_.isInstanceOf[StringType])) + assert(mapType.existsRecursively(_.isInstanceOf[MapType])) + assert(!mapType.existsRecursively(_.isInstanceOf[IntegerType])) + + val arrayType = ArrayType(mapType) + assert(arrayType.existsRecursively(_.isInstanceOf[LongType])) + assert(arrayType.existsRecursively(_.isInstanceOf[StructType])) + assert(arrayType.existsRecursively(_.isInstanceOf[StringType])) + assert(arrayType.existsRecursively(_.isInstanceOf[MapType])) + assert(arrayType.existsRecursively(_.isInstanceOf[ArrayType])) + assert(!arrayType.existsRecursively(_.isInstanceOf[IntegerType])) + } + + def checkDataTypeFromJson(dataType: DataType): Unit = { + test(s"from Json - $dataType") { + assert(DataType.fromJson(dataType.json) === dataType) + } + } + + def checkDataTypeFromDDL(dataType: DataType): Unit = { + test(s"from DDL - $dataType") { + val parsed = StructType.fromDDL(s"a ${dataType.sql}") + val expected = new StructType().add("a", dataType) + assert(parsed.sameType(expected)) + } + } + + checkDataTypeFromJson(NullType) + + checkDataTypeFromJson(BooleanType) + checkDataTypeFromDDL(BooleanType) + + checkDataTypeFromJson(ByteType) + checkDataTypeFromDDL(ByteType) + + checkDataTypeFromJson(ShortType) + checkDataTypeFromDDL(ShortType) + + checkDataTypeFromJson(IntegerType) + checkDataTypeFromDDL(IntegerType) + + checkDataTypeFromJson(LongType) + checkDataTypeFromDDL(LongType) + + checkDataTypeFromJson(FloatType) + checkDataTypeFromDDL(FloatType) + + checkDataTypeFromJson(DoubleType) + checkDataTypeFromDDL(DoubleType) + + checkDataTypeFromJson(DecimalType(10, 5)) + checkDataTypeFromDDL(DecimalType(10, 5)) + + checkDataTypeFromJson(DecimalType.SYSTEM_DEFAULT) + checkDataTypeFromDDL(DecimalType.SYSTEM_DEFAULT) + + checkDataTypeFromJson(DateType) + checkDataTypeFromDDL(DateType) + + checkDataTypeFromJson(TimestampType) + checkDataTypeFromDDL(TimestampType) + + checkDataTypeFromJson(StringType) + checkDataTypeFromDDL(StringType) + + checkDataTypeFromJson(BinaryType) + checkDataTypeFromDDL(BinaryType) + + checkDataTypeFromJson(ArrayType(DoubleType, true)) + checkDataTypeFromDDL(ArrayType(DoubleType, true)) + + checkDataTypeFromJson(ArrayType(StringType, false)) + checkDataTypeFromDDL(ArrayType(StringType, false)) + + checkDataTypeFromJson(MapType(IntegerType, StringType, true)) + checkDataTypeFromDDL(MapType(IntegerType, StringType, true)) + + checkDataTypeFromJson(MapType(IntegerType, ArrayType(DoubleType), false)) + checkDataTypeFromDDL(MapType(IntegerType, ArrayType(DoubleType), false)) + + val metadata = new MetadataBuilder() + .putString("name", "age") + .build() + val structType = StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", ArrayType(DoubleType), nullable = false), + StructField("c", DoubleType, nullable = false, metadata))) + checkDataTypeFromJson(structType) + checkDataTypeFromDDL(structType) + + test("fromJson throws an exception when given type string is invalid") { + var message = intercept[IllegalArgumentException] { + DataType.fromJson(""""abcd"""") + }.getMessage + assert(message.contains( + "Failed to convert the JSON string 'abcd' to a data type.")) + + message = intercept[IllegalArgumentException] { + DataType.fromJson("""{"abcd":"a"}""") + }.getMessage + assert(message.contains( + """Failed to convert the JSON string '{"abcd":"a"}' to a data type""")) + + message = intercept[IllegalArgumentException] { + DataType.fromJson("""{"fields": [{"a":123}], "type": "struct"}""") + }.getMessage + assert(message.contains( + """Failed to convert the JSON string '{"a":123}' to a field.""")) + + // Malformed JSON string + message = intercept[JsonParseException] { + DataType.fromJson("abcd") + }.getMessage + assert(message.contains("Unrecognized token 'abcd'")) + } + + def checkDefaultSize(dataType: DataType, expectedDefaultSize: Int): Unit = { + test(s"Check the default size of $dataType") { + assert(dataType.defaultSize === expectedDefaultSize) + } + } + + checkDefaultSize(NullType, 1) + checkDefaultSize(BooleanType, 1) + checkDefaultSize(ByteType, 1) + checkDefaultSize(ShortType, 2) + checkDefaultSize(IntegerType, 4) + checkDefaultSize(LongType, 8) + checkDefaultSize(FloatType, 4) + checkDefaultSize(DoubleType, 8) + checkDefaultSize(DecimalType(10, 5), 8) + checkDefaultSize(DecimalType.SYSTEM_DEFAULT, 16) + checkDefaultSize(DateType, 4) + checkDefaultSize(TimestampType, 8) + checkDefaultSize(StringType, 20) + checkDefaultSize(BinaryType, 100) + checkDefaultSize(ArrayType(DoubleType, true), 8) + checkDefaultSize(ArrayType(StringType, false), 20) + checkDefaultSize(MapType(IntegerType, StringType, true), 24) + checkDefaultSize(MapType(IntegerType, ArrayType(DoubleType), false), 12) + checkDefaultSize(structType, 20) + + def checkEqualsIgnoreCompatibleNullability( + from: DataType, + to: DataType, + expected: Boolean): Unit = { + val testName = + s"equalsIgnoreCompatibleNullability: (from: $from, to: $to)" + test(testName) { + assert(DataType.equalsIgnoreCompatibleNullability(from, to) === expected) + } + } + + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = true), + to = ArrayType(DoubleType, containsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(DoubleType, containsNull = false), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(DoubleType, containsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = true), + to = ArrayType(DoubleType, containsNull = false), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(StringType, containsNull = false), + expected = false) + + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = true), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = false), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = true), + to = MapType(StringType, DoubleType, valueContainsNull = false), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, ArrayType(IntegerType, true), valueContainsNull = true), + to = MapType(StringType, ArrayType(IntegerType, false), valueContainsNull = true), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, ArrayType(IntegerType, false), valueContainsNull = true), + to = MapType(StringType, ArrayType(IntegerType, true), valueContainsNull = true), + expected = true) + + + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = StructType( + StructField("a", StringType, nullable = false) :: + StructField("b", StringType, nullable = true) :: Nil), + to = StructType( + StructField("a", StringType, nullable = false) :: + StructField("b", StringType, nullable = false) :: Nil), + expected = false) + + def checkCatalogString(dt: DataType): Unit = { + test(s"catalogString: $dt") { + val dt2 = CatalystSqlParser.parseDataType(dt.catalogString) + assert(dt === dt2) + } + } + def createStruct(n: Int): StructType = new StructType(Array.tabulate(n) { + i => StructField(s"col$i", IntegerType, nullable = true) + }) + + checkCatalogString(BooleanType) + checkCatalogString(ByteType) + checkCatalogString(ShortType) + checkCatalogString(IntegerType) + checkCatalogString(LongType) + checkCatalogString(FloatType) + checkCatalogString(DoubleType) + checkCatalogString(DecimalType(10, 5)) + checkCatalogString(BinaryType) + checkCatalogString(StringType) + checkCatalogString(DateType) + checkCatalogString(TimestampType) + checkCatalogString(createStruct(4)) + checkCatalogString(createStruct(40)) + checkCatalogString(ArrayType(IntegerType)) + checkCatalogString(ArrayType(createStruct(40))) + checkCatalogString(MapType(IntegerType, StringType)) + checkCatalogString(MapType(IntegerType, createStruct(40))) + + def checkEqualsStructurally(from: DataType, to: DataType, expected: Boolean): Unit = { + val testName = s"equalsStructurally: (from: $from, to: $to)" + test(testName) { + assert(DataType.equalsStructurally(from, to) === expected) + } + } + + checkEqualsStructurally(BooleanType, BooleanType, true) + checkEqualsStructurally(IntegerType, IntegerType, true) + checkEqualsStructurally(IntegerType, LongType, false) + checkEqualsStructurally(ArrayType(IntegerType, true), ArrayType(IntegerType, true), true) + checkEqualsStructurally(ArrayType(IntegerType, true), ArrayType(IntegerType, false), false) + + checkEqualsStructurally( + new StructType().add("f1", IntegerType), + new StructType().add("f2", IntegerType), + true) + checkEqualsStructurally( + new StructType().add("f1", IntegerType), + new StructType().add("f2", IntegerType, false), + false) + + checkEqualsStructurally( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType)), + new StructType().add("f2", IntegerType).add("g", new StructType().add("f1", StringType)), + true) + checkEqualsStructurally( + new StructType().add("f1", IntegerType).add("f", new StructType().add("f2", StringType, false)), + new StructType().add("f2", IntegerType).add("g", new StructType().add("f1", StringType)), + false) +} 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..23aa60258be9 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; /** @@ -88,8 +88,8 @@ public static void populate(WritableColumnVector col, InternalRow row, int field } } else if (t instanceof CalendarIntervalType) { CalendarInterval c = (CalendarInterval)row.get(fieldIdx, t); - col.getChild(0).putInts(0, capacity, c.months); - col.getChild(1).putLongs(0, capacity, c.microseconds); + col.getChild(0).putInts(0, capacity, c.months()); + col.getChild(1).putLongs(0, capacity, c.microseconds()); } else if (t instanceof DateType) { col.putInts(0, capacity, row.getInt(fieldIdx)); } else if (t instanceof TimestampType) { @@ -164,8 +164,8 @@ private static void appendValue(WritableColumnVector dst, DataType t, Object o) } else if (t instanceof CalendarIntervalType) { CalendarInterval c = (CalendarInterval)o; dst.appendStruct(false); - dst.getChild(0).appendInt(c.months); - dst.getChild(1).appendLong(c.microseconds); + dst.getChild(0).appendInt(c.months()); + dst.getChild(1).appendLong(c.microseconds()); } else if (t instanceof DateType) { dst.appendInt(DateTimeUtils.fromJavaDate((Date)o)); } else { 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 4e4242fe8d9b..7776261d7881 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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.sql.vectorized.ColumnarRow; import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index ad99b450a480..38694f3d1679 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/core/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.InterfaceStability; +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/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java index 72a192d089b9..4cdbe7f10ac5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java @@ -19,7 +19,6 @@ import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index f2f227959002..6c0025bbaafe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/core/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/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 2ec236fc75ef..3c92ffce7deb 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.streaming.DataStreamWriter import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel -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 b161651c4e6a..42c32ceb867e 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 @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark 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 7f65e3ea9dd5..84160364ced8 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 @@ -24,7 +24,7 @@ import org.apache.commons.lang3.StringUtils 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 4aba76cad367..df3a274c15c7 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 90e1766c4d9f..229df9db97c3 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 @@ -25,7 +25,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql.streaming.{ProcessingTime, 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 acca9572cb14..6caf8416ec18 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 @@ -2972,14 +2972,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 @@ -3028,14 +3028,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. * @@ -3073,7 +3073,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 a033575d3d38..09164b29d08d 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 @@ -24,7 +24,7 @@ import scala.concurrent.duration.Duration import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.InterfaceStability -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/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala index 2a0b2b85e10a..6fc68a673814 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.sql.types.CalendarInterval /** * Window frame testing for DataFrame API. 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 237412aa692e..d76d1931bec4 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 @@ -25,7 +25,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 640affc10ee5..b976ecc4c212 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 @@ -1467,7 +1467,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 ))) @@ -1491,8 +1491,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 f57f07b49826..fb54226be19c 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 @@ -35,7 +35,6 @@ import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.CalendarInterval 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 61f9179042fe..ac75f60c14a2 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.hive.test.TestHiveSingleton -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 828c18a770c8..3f37dff4a1d5 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 @@ -39,7 +39,6 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf 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 b6131d3ea478d56309ef6773b5682fb9a09ab6e8 Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Mon, 1 Jul 2019 14:13:38 +0530 Subject: [PATCH 3/5] SPARK-24695: To move calendar Interval class --- .../expressions/codegen/UnsafeWriter.java | 4 +- .../spark/sql/types/CalendarInterval.java | 360 ++++++++++++++++++ .../spark/sql/types/CalendarInterval.scala | 315 --------------- .../sql/types/CalendarIntervalSuite.java | 268 +++++++++++++ .../sql/types/CalendarIntervalSuite.scala | 226 ----------- 5 files changed, 630 insertions(+), 543 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala create mode 100644 sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala 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 f91bb183f340..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 @@ -135,8 +135,8 @@ public final void write(int ordinal, CalendarInterval input) { grow(16); // Write the months and microseconds fields of Interval to the variable length portion. - Platform.putLong(getBuffer(), cursor(), input.months()); - Platform.putLong(getBuffer(), cursor() + 8, input.microseconds()); + Platform.putLong(getBuffer(), cursor(), input.months); + Platform.putLong(getBuffer(), cursor() + 8, input.microseconds); setOffsetAndSize(ordinal, 16); 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 new file mode 100644 index 000000000000..9484f6e52817 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java @@ -0,0 +1,360 @@ +/* + * 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 java.io.Serializable; +import java.util.Locale; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * The internal representation of interval type. + */ +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; + public static final long MICROS_PER_MINUTE = MICROS_PER_SECOND * 60; + public static final long MICROS_PER_HOUR = MICROS_PER_MINUTE * 60; + public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; + public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; + + /** + * A function to generate regex which matches interval string's unit part like "3 years". + * + * First, we can leave out some units in interval string, and we only care about the value of + * unit, so here we use non-capturing group to wrap the actual regex. + * At the beginning of the actual regex, we should match spaces before the unit part. + * Next is the number part, starts with an optional "-" to represent negative value. We use + * capturing group to wrap this part as we need the value later. + * Finally is the unit name, ends with an optional "s". + */ + private static String unitRegex(String unit) { + return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?"; + } + + private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + + unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + + unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")); + + private static Pattern yearMonthPattern = + Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); + + private static Pattern dayTimePattern = + Pattern.compile("^(?:['|\"])?([+|-])?((\\d+) )?(\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); + + private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); + + private static long toLong(String s) { + if (s == null) { + return 0; + } else { + return Long.parseLong(s); + } + } + + /** + * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. + * This method is case-sensitive and all characters in the input string should be in lower case. + */ + public static CalendarInterval fromString(String s) { + if (s == null) { + return null; + } + s = s.trim(); + Matcher m = p.matcher(s); + if (!m.matches() || s.equals("interval")) { + return null; + } else { + long months = toLong(m.group(1)) * 12 + toLong(m.group(2)); + long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK; + microseconds += toLong(m.group(4)) * MICROS_PER_DAY; + microseconds += toLong(m.group(5)) * MICROS_PER_HOUR; + microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE; + microseconds += toLong(m.group(7)) * MICROS_PER_SECOND; + microseconds += toLong(m.group(8)) * MICROS_PER_MILLI; + microseconds += toLong(m.group(9)); + return new CalendarInterval((int) months, microseconds); + } + } + + /** + * Convert a string to CalendarInterval. Unlike fromString, this method is case-insensitive and + * will throw IllegalArgumentException when the input string is not a valid interval. + * + * @throws IllegalArgumentException if the string is not a valid internal. + */ + public static CalendarInterval fromCaseInsensitiveString(String s) { + if (s == null || s.trim().isEmpty()) { + throw new IllegalArgumentException("Interval cannot be null or blank."); + } + String sInLowerCase = s.trim().toLowerCase(Locale.ROOT); + String interval = + sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; + CalendarInterval cal = fromString(interval); + if (cal == null) { + throw new IllegalArgumentException("Invalid interval: " + s); + } + return cal; + } + + public static long toLongWithRange(String fieldName, + String s, long minValue, long maxValue) throws IllegalArgumentException { + long result = 0; + if (s != null) { + result = Long.parseLong(s); + if (result < minValue || result > maxValue) { + throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]", + fieldName, result, minValue, maxValue)); + } + } + return result; + } + + /** + * Parse YearMonth string in form: [-]YYYY-MM + * + * adapted from HiveIntervalYearMonth.valueOf + */ + public static CalendarInterval fromYearMonthString(String s) throws IllegalArgumentException { + CalendarInterval result = null; + if (s == null) { + throw new IllegalArgumentException("Interval year-month string was null"); + } + s = s.trim(); + Matcher m = yearMonthPattern.matcher(s); + if (!m.matches()) { + throw new IllegalArgumentException( + "Interval string does not match year-month format of 'y-m': " + s); + } else { + try { + int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; + int years = (int) toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE); + int months = (int) toLongWithRange("month", m.group(3), 0, 11); + result = new CalendarInterval(sign * (years * 12 + months), 0); + } catch (Exception e) { + throw new IllegalArgumentException( + "Error parsing interval year-month string: " + e.getMessage(), e); + } + } + return result; + } + + /** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf + */ + public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException { + CalendarInterval result = null; + if (s == null) { + throw new IllegalArgumentException("Interval day-time string was null"); + } + s = s.trim(); + Matcher m = dayTimePattern.matcher(s); + if (!m.matches()) { + throw new IllegalArgumentException( + "Interval string does not match day-time format of 'd h:m:s.n': " + s); + } else { + try { + int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; + long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3), + 0, Integer.MAX_VALUE); + long hours = toLongWithRange("hour", m.group(4), 0, 23); + long minutes = toLongWithRange("minute", m.group(5), 0, 59); + long seconds = toLongWithRange("second", m.group(6), 0, 59); + // Hive allow nanosecond precision interval + long nanos = toLongWithRange("nanosecond", m.group(8), 0L, 999999999L); + result = new CalendarInterval(0, sign * ( + days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + + seconds * MICROS_PER_SECOND + nanos / 1000L)); + } catch (Exception e) { + throw new IllegalArgumentException( + "Error parsing interval day-time string: " + e.getMessage(), e); + } + } + return result; + } + + public static CalendarInterval fromSingleUnitString(String unit, String s) + throws IllegalArgumentException { + + CalendarInterval result = null; + if (s == null) { + throw new IllegalArgumentException(String.format("Interval %s string was null", unit)); + } + s = s.trim(); + Matcher m = quoteTrimPattern.matcher(s); + if (!m.matches()) { + throw new IllegalArgumentException( + "Interval string does not match day-time format of 'd h:m:s.n': " + s); + } else { + try { + switch (unit) { + case "year": + int year = (int) toLongWithRange("year", m.group(1), + Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); + result = new CalendarInterval(year * 12, 0L); + break; + case "month": + int month = (int) toLongWithRange("month", m.group(1), + Integer.MIN_VALUE, Integer.MAX_VALUE); + result = new CalendarInterval(month, 0L); + break; + case "week": + long week = toLongWithRange("week", m.group(1), + Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); + result = new CalendarInterval(0, week * MICROS_PER_WEEK); + break; + case "day": + long day = toLongWithRange("day", m.group(1), + Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); + result = new CalendarInterval(0, day * MICROS_PER_DAY); + break; + case "hour": + long hour = toLongWithRange("hour", m.group(1), + Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); + result = new CalendarInterval(0, hour * MICROS_PER_HOUR); + break; + case "minute": + long minute = toLongWithRange("minute", m.group(1), + Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); + result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); + break; + case "second": { + long micros = parseSecondNano(m.group(1)); + result = new CalendarInterval(0, micros); + break; + } + case "millisecond": + long millisecond = toLongWithRange("millisecond", m.group(1), + Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); + result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); + break; + case "microsecond": { + long micros = Long.parseLong(m.group(1)); + result = new CalendarInterval(0, micros); + break; + } + } + } catch (Exception e) { + throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); + } + } + return result; + } + + /** + * Parse second_nano string in ss.nnnnnnnnn format to microseconds + */ + public static long parseSecondNano(String secondNano) throws IllegalArgumentException { + String[] parts = secondNano.split("\\."); + if (parts.length == 1) { + return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND, + Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND; + + } else if (parts.length == 2) { + long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0], + Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND); + long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L); + return seconds * MICROS_PER_SECOND + nanos / 1000L; + + } else { + throw new IllegalArgumentException( + "Interval string does not match second-nano format of ss.nnnnnnnnn"); + } + } + + public final int months; + public final long microseconds; + + public long milliseconds() { + return this.microseconds / MICROS_PER_MILLI; + } + + public CalendarInterval(int months, long microseconds) { + this.months = months; + this.microseconds = microseconds; + } + + public CalendarInterval add(CalendarInterval that) { + int months = this.months + that.months; + long microseconds = this.microseconds + that.microseconds; + return new CalendarInterval(months, microseconds); + } + + public CalendarInterval subtract(CalendarInterval that) { + int months = this.months - that.months; + long microseconds = this.microseconds - that.microseconds; + return new CalendarInterval(months, microseconds); + } + + public CalendarInterval negate() { + return new CalendarInterval(-this.months, -this.microseconds); + } + + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other == null || !(other instanceof CalendarInterval)) return false; + + CalendarInterval o = (CalendarInterval) other; + return this.months == o.months && this.microseconds == o.microseconds; + } + + @Override + public int hashCode() { + return 31 * months + (int) microseconds; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("interval"); + + if (months != 0) { + appendUnit(sb, months / 12, "year"); + appendUnit(sb, months % 12, "month"); + } + + if (microseconds != 0) { + long rest = microseconds; + appendUnit(sb, rest / MICROS_PER_WEEK, "week"); + rest %= MICROS_PER_WEEK; + appendUnit(sb, rest / MICROS_PER_DAY, "day"); + rest %= MICROS_PER_DAY; + appendUnit(sb, rest / MICROS_PER_HOUR, "hour"); + rest %= MICROS_PER_HOUR; + appendUnit(sb, rest / MICROS_PER_MINUTE, "minute"); + rest %= MICROS_PER_MINUTE; + appendUnit(sb, rest / MICROS_PER_SECOND, "second"); + rest %= MICROS_PER_SECOND; + appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond"); + rest %= MICROS_PER_MILLI; + appendUnit(sb, rest, "microsecond"); + } else if (months == 0) { + sb.append(" 0 microseconds"); + } + + return sb.toString(); + } + + private void appendUnit(StringBuilder sb, long value, String unit) { + if (value != 0) { + sb.append(' ').append(value).append(' ').append(unit).append('s'); + } + } +} \ No newline at end of file diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala deleted file mode 100644 index b19f86f78720..000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarInterval.scala +++ /dev/null @@ -1,315 +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 java.io.Serializable -import java.util.Locale -import java.util.regex.Pattern - - -/** - * The internal representation of interval type. - */ - -object CalendarInterval { - val MICROS_PER_MILLI = 1000L - val MICROS_PER_SECOND: Long = MICROS_PER_MILLI * 1000 - val MICROS_PER_MINUTE: Long = MICROS_PER_SECOND * 60 - val MICROS_PER_HOUR: Long = MICROS_PER_MINUTE * 60 - val MICROS_PER_DAY: Long = MICROS_PER_HOUR * 24 - val MICROS_PER_WEEK: Long = MICROS_PER_DAY * 7 - - /** - * A function to generate regex which matches interval string's unit part like "3 years". - * - * First, we can leave out some units in interval string, and we only care about the value of - * unit, so here we use non-capturing group to wrap the actual regex. - * At the beginning of the actual regex, we should match spaces before the unit part. - * Next is the number part, starts with an optional "-" to represent negative value. We use - * capturing group to wrap this part as we need the value later. - * Finally is the unit name, ends with an optional "s". - */ - private def unitRegex(unit: String) = "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?" - - private val p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")) - private val yearMonthPattern = Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$") - private val dayTimePattern = Pattern.compile("^(?:['|\"])?([+|-])?((\\d+) )?" + - "(\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$") - private val quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$") - - private def toLong(s: String) = if (s == null) 0L - else s.toLong - - /** - * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. - * This method is case-sensitive and all characters in the input string should be in lower case. - */ - def fromString(s: String): CalendarInterval = { - if (s == null) return null - val m = p.matcher(s.trim) - if (!m.matches || s == "interval") null - else { - val months = toLong(m.group(1)) * 12 + toLong(m.group(2)) - var microseconds = toLong(m.group(3)) * MICROS_PER_WEEK - microseconds += toLong(m.group(4)) * MICROS_PER_DAY - microseconds += toLong(m.group(5)) * MICROS_PER_HOUR - microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE - microseconds += toLong(m.group(7)) * MICROS_PER_SECOND - microseconds += toLong(m.group(8)) * MICROS_PER_MILLI - microseconds += toLong(m.group(9)) - new CalendarInterval(months.toInt, microseconds) - } - } - - /** - * Convert a string to CalendarInterval. Unlike fromString, this method is case-insensitive and - * will throw IllegalArgumentException when the input string is not a valid interval. - * - * @throws IllegalArgumentException if the string is not a valid internal. - */ - def fromCaseInsensitiveString(s: String): CalendarInterval = { - if (s == null || s.trim.isEmpty) { - throw new IllegalArgumentException("Interval cannot be null or blank.") - } - val sInLowerCase = s.trim.toLowerCase(Locale.ROOT) - val interval = if (sInLowerCase.startsWith("interval ")) sInLowerCase - else "interval " + sInLowerCase - val cal = fromString(interval) - if (cal == null) throw new IllegalArgumentException("Invalid interval: " + s) - cal - } - - @throws[IllegalArgumentException] - def toLongWithRange(fieldName: String, s: String, minValue: Long, maxValue: Long): Long = { - var result = 0L - if (s != null) { - result = s.toLong - if (result < minValue || result > maxValue) { - throw new IllegalArgumentException( - s"${fieldName} ${result} outside range [${minValue}, ${maxValue}]") - } - } - result - } - - /** - * Parse YearMonth string in form: [-]YYYY-MM - * - * adapted from HiveIntervalYearMonth.valueOf - */ - @throws[IllegalArgumentException] - def fromYearMonthString(s: String): CalendarInterval = { - var result : CalendarInterval = null - if (s == null) { - throw new IllegalArgumentException("Interval year-month string was null") - } - - val m = yearMonthPattern.matcher(s.trim) - if (!m.matches) { - throw new IllegalArgumentException( - "Interval string does not match year-month format of 'y-m': " + s) - } - else try { - val sign = if (m.group(1) != null && m.group(1) == "-") -1 - else 1 - val years = toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE).toInt - val months = toLongWithRange("month", m.group(3), 0, 11).toInt - result = new CalendarInterval(sign * (years * 12 + months), 0) - } catch { - case e: Exception => - throw new IllegalArgumentException( - "Error parsing interval year-month string: " + e.getMessage, e) - } - result - } - - /** - * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn - * - * adapted from HiveIntervalDayTime.valueOf - */ - @throws[IllegalArgumentException] - def fromDayTimeString(s: String): CalendarInterval = { - var result : CalendarInterval = null - if (s == null) throw new IllegalArgumentException("Interval day-time string was null") - val m = dayTimePattern.matcher(s.trim) - if (!m.matches) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s) - } - else try { - val sign = if (m.group(1) != null && m.group(1) == "-") -1 - else 1 - val days = if (m.group(2) == null) 0 - else toLongWithRange("day", m.group(3), 0, Integer.MAX_VALUE) - val hours = toLongWithRange("hour", m.group(4), 0, 23) - val minutes = toLongWithRange("minute", m.group(5), 0, 59) - val seconds = toLongWithRange("second", m.group(6), 0, 59) - // Hive allow nanosecond precision interval - val nanos = toLongWithRange("nanosecond", m.group(8), 0L, 999999999L) - result = new CalendarInterval(0, sign * (days * MICROS_PER_DAY + - hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + - seconds * MICROS_PER_SECOND + nanos / 1000L)) - } catch { - case e: Exception => - throw new IllegalArgumentException( - "Error parsing interval day-time string: " + e.getMessage, e) - } - result - } - - @throws[IllegalArgumentException] - def fromSingleUnitString(unit: String, s: String): CalendarInterval = { - var result : CalendarInterval = null - if (s == null) { - throw new IllegalArgumentException(String.format("Interval %s string was null", unit)) - } - - val m = quoteTrimPattern.matcher(s.trim) - if (!m.matches) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s) - } - else try - unit match { - case "year" => - val year = toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12).toInt - result = new CalendarInterval(year * 12, 0L) - case "month" => - val month = toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE).toInt - result = new CalendarInterval(month, 0L) - case "week" => - val week = toLongWithRange("week", m.group(1), - Long.MinValue / MICROS_PER_WEEK, Long.MaxValue / MICROS_PER_WEEK) - result = new CalendarInterval(0, week * MICROS_PER_WEEK) - case "day" => - val day = toLongWithRange("day", m.group(1), - Long.MinValue / MICROS_PER_DAY, Long.MaxValue / MICROS_PER_DAY) - result = new CalendarInterval(0, day * MICROS_PER_DAY) - case "hour" => - val hour = toLongWithRange("hour", m.group(1), - Long.MinValue / MICROS_PER_HOUR, Long.MaxValue / MICROS_PER_HOUR) - result = new CalendarInterval(0, hour * MICROS_PER_HOUR) - case "minute" => - val minute = toLongWithRange("minute", m.group(1), - Long.MinValue / MICROS_PER_MINUTE, Long.MaxValue / MICROS_PER_MINUTE) - result = new CalendarInterval(0, minute * MICROS_PER_MINUTE) - case "second" => - val micros = parseSecondNano(m.group(1)) - result = new CalendarInterval(0, micros) - case "millisecond" => - val millisecond = toLongWithRange("millisecond", m.group(1), - Long.MinValue / MICROS_PER_MILLI, Long.MaxValue / MICROS_PER_MILLI) - result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI) - case "microsecond" => - val micros = m.group(1).toLong - result = new CalendarInterval(0, micros) - } - catch { - case e: Exception => - throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage, e) - } - result - } - - /** - * Parse second_nano string in ss.nnnnnnnnn format to microseconds - */ - @throws[IllegalArgumentException] - def parseSecondNano(secondNano: String): Long = { - val parts = secondNano.split("\\.") - if (parts.length == 1) { - toLongWithRange("second", parts(0), Long.MinValue/ MICROS_PER_SECOND, - Long.MaxValue / MICROS_PER_SECOND) * MICROS_PER_SECOND - } - else if (parts.length == 2) { - val seconds = if (parts(0) == "") 0L - else { - toLongWithRange("second", parts(0), Long.MinValue / MICROS_PER_SECOND, - Long.MaxValue / MICROS_PER_SECOND) - } - val nanos = toLongWithRange("nanosecond", parts(1), 0L, 999999999L) - seconds * MICROS_PER_SECOND + nanos / 1000L - } - else { - throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn") - } - } -} - -final class CalendarInterval(val months: Int, val microseconds: Long) extends Serializable { - def milliseconds: Long = this.microseconds / CalendarInterval.MICROS_PER_MILLI - - def add(that: CalendarInterval): CalendarInterval = { - val months = this.months + that.months - val microseconds = this.microseconds + that.microseconds - new CalendarInterval(months, microseconds) - } - - def subtract(that: CalendarInterval): CalendarInterval = { - val months = this.months - that.months - val microseconds = this.microseconds - that.microseconds - new CalendarInterval(months, microseconds) - } - - def negate : CalendarInterval = new CalendarInterval(-this.months, -this.microseconds) - - override def equals(other: Any): Boolean = { - if (this equals other) return true - if (other == null || !other.isInstanceOf[CalendarInterval]) return false - val o = other.asInstanceOf[CalendarInterval] - this.months == o.months && this.microseconds == o.microseconds - } - - override def hashCode: Int = 31 * months + microseconds.toInt - - override def toString: String = { - val sb = new StringBuilder("interval") - if (months != 0) { - appendUnit(sb, months / 12, "year") - appendUnit(sb, months % 12, "month") - } - if (microseconds != 0) { - var rest = microseconds - appendUnit(sb, rest / CalendarInterval.MICROS_PER_WEEK, "week") - rest %= CalendarInterval.MICROS_PER_WEEK - appendUnit(sb, rest / CalendarInterval.MICROS_PER_DAY, "day") - rest %= CalendarInterval.MICROS_PER_DAY - appendUnit(sb, rest / CalendarInterval.MICROS_PER_HOUR, "hour") - rest %= CalendarInterval.MICROS_PER_HOUR - appendUnit(sb, rest / CalendarInterval.MICROS_PER_MINUTE, "minute") - rest %= CalendarInterval.MICROS_PER_MINUTE - appendUnit(sb, rest / CalendarInterval.MICROS_PER_SECOND, "second") - rest %= CalendarInterval.MICROS_PER_SECOND - appendUnit(sb, rest / CalendarInterval.MICROS_PER_MILLI, "millisecond") - rest %= CalendarInterval.MICROS_PER_MILLI - appendUnit(sb, rest, "microsecond") - } - else if (months == 0) sb.append(" 0 microseconds") - sb.toString - } - - private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { - if (value != 0) sb.append(' ').append(value).append(' ').append(unit).append('s') - } -} 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/types/CalendarIntervalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala deleted file mode 100644 index a668d5f5c5d0..000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/CalendarIntervalSuite.scala +++ /dev/null @@ -1,226 +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.Assert._ -import org.junit.Test - -import org.apache.spark.sql.types.CalendarInterval._ - -object CalendarIntervalSuite { - private def testSingleUnit(unit: String, number: Int, months: Int, microseconds: Long): Unit = { - val input1 = "interval " + number + " " + unit - val input2 = "interval " + number + " " + unit + "s" - val result = new CalendarInterval(months, microseconds) - assertEquals(fromString(input1), result) - assertEquals(fromString(input2), result) - } -} - -class CalendarIntervalSuite { - @Test def equalsTest(): Unit = { - val i1 = new CalendarInterval(3, 123) - val i2 = new CalendarInterval(3, 321) - val i3 = new CalendarInterval(1, 123) - val i4 = new CalendarInterval(3, 123) - assertNotSame(i1, i2) - assertNotSame(i1, i3) - assertNotSame(i2, i3) - assertEquals(i1, i4) - } - - @Test def toStringTest(): Unit = { - var i : CalendarInterval = null - 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 def fromStringTest(): Unit = { - CalendarIntervalSuite.testSingleUnit("year", 3, 36, 0) - CalendarIntervalSuite.testSingleUnit("month", 3, 3, 0) - CalendarIntervalSuite.testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK) - CalendarIntervalSuite.testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY) - CalendarIntervalSuite.testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR) - CalendarIntervalSuite.testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE) - CalendarIntervalSuite.testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND) - CalendarIntervalSuite.testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI) - CalendarIntervalSuite.testSingleUnit("microsecond", 3, 0, 3) - var input : String = null - input = "interval -5 years 23 month" - val 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 def fromCaseInsensitiveStringTest(): Unit = { - for (input <- Array[String]("5 MINUTES", "5 minutes", "5 Minutes")) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1000000)) - } - for (input <- Array[String](null, "", " ")) { - try { - fromCaseInsensitiveString(input) - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("cannot be null or blank")) - } - } - for (input <- Array[String]("interval", "interval1 day", "foo", "foo 1 day")) { - try { - fromCaseInsensitiveString(input) - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("Invalid interval")) - } - } - } - - @Test def fromYearMonthStringTest(): Unit = { - var input : String = null - var i : CalendarInterval = null - 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 { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("month 15 outside range")) - } - } - - @Test def fromDayTimeStringTest(): Unit = { - var input : String = null - var i : CalendarInterval = null - 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 { - case e: IllegalArgumentException => - 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 { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("not match day-time format")) - } - } - - @Test def fromSingleUnitStringTest(): Unit = { - var input : String = null - var i : CalendarInterval = null - 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 { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("outside range")) - } - try { - input = String.valueOf(Long.MaxValue / MICROS_PER_HOUR + 1) - fromSingleUnitString("hour", input) - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assertTrue(e.getMessage.contains("outside range")) - } - } - - @Test def addTest(): Unit = { - var input = "interval 3 month 1 hour" - var input2 = "interval 2 month 100 hour" - var interval = fromString(input) - var 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 def subtractTest(): Unit = { - var input = "interval 3 month 1 hour" - var input2 = "interval 2 month 100 hour" - var interval = fromString(input) - var 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)) - } -} From 799a2cf48984d9413d9ff31e67f85cee9632d68e Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Mon, 1 Jul 2019 14:39:25 +0530 Subject: [PATCH 4/5] SPARK-24695: To remove extra spaces --- .../spark/sql/types/CalendarInterval.java | 51 +++++++++---------- .../spark/sql/vectorized/ColumnVector.java | 1 + 2 files changed, 26 insertions(+), 26 deletions(-) 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 9484f6e52817..710066a6be6a 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 @@ -48,14 +48,14 @@ private static String unitRegex(String unit) { } private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")); + unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + + unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond")); private static Pattern yearMonthPattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); + Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); private static Pattern dayTimePattern = - Pattern.compile("^(?:['|\"])?([+|-])?((\\d+) )?(\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); + Pattern.compile("^(?:['|\"])?([+|-])?((\\d+) )?(\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); @@ -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); @@ -113,13 +112,13 @@ public static CalendarInterval fromCaseInsensitiveString(String s) { } public static long toLongWithRange(String fieldName, - String s, long minValue, long maxValue) throws IllegalArgumentException { + String s, long minValue, long maxValue) throws IllegalArgumentException { long result = 0; if (s != null) { result = Long.parseLong(s); if (result < minValue || result > maxValue) { throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]", - fieldName, result, minValue, maxValue)); + fieldName, result, minValue, maxValue)); } } return result; @@ -139,7 +138,7 @@ public static CalendarInterval fromYearMonthString(String s) throws IllegalArgum Matcher m = yearMonthPattern.matcher(s); if (!m.matches()) { throw new IllegalArgumentException( - "Interval string does not match year-month format of 'y-m': " + s); + "Interval string does not match year-month format of 'y-m': " + s); } else { try { int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; @@ -148,7 +147,7 @@ public static CalendarInterval fromYearMonthString(String s) throws IllegalArgum result = new CalendarInterval(sign * (years * 12 + months), 0); } catch (Exception e) { throw new IllegalArgumentException( - "Error parsing interval year-month string: " + e.getMessage(), e); + "Error parsing interval year-month string: " + e.getMessage(), e); } } return result; @@ -168,23 +167,23 @@ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumen Matcher m = dayTimePattern.matcher(s); if (!m.matches()) { throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); + "Interval string does not match day-time format of 'd h:m:s.n': " + s); } else { try { int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3), - 0, Integer.MAX_VALUE); + 0, Integer.MAX_VALUE); long hours = toLongWithRange("hour", m.group(4), 0, 23); long minutes = toLongWithRange("minute", m.group(5), 0, 59); long seconds = toLongWithRange("second", m.group(6), 0, 59); // Hive allow nanosecond precision interval long nanos = toLongWithRange("nanosecond", m.group(8), 0L, 999999999L); result = new CalendarInterval(0, sign * ( - days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + - seconds * MICROS_PER_SECOND + nanos / 1000L)); + days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + + seconds * MICROS_PER_SECOND + nanos / 1000L)); } catch (Exception e) { throw new IllegalArgumentException( - "Error parsing interval day-time string: " + e.getMessage(), e); + "Error parsing interval day-time string: " + e.getMessage(), e); } } return result; @@ -201,38 +200,38 @@ public static CalendarInterval fromSingleUnitString(String unit, String s) Matcher m = quoteTrimPattern.matcher(s); if (!m.matches()) { throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); + "Interval string does not match day-time format of 'd h:m:s.n': " + s); } else { try { switch (unit) { case "year": int year = (int) toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); + Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); result = new CalendarInterval(year * 12, 0L); break; case "month": int month = (int) toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE); + Integer.MIN_VALUE, Integer.MAX_VALUE); result = new CalendarInterval(month, 0L); break; case "week": long week = toLongWithRange("week", m.group(1), - Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); + Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); result = new CalendarInterval(0, week * MICROS_PER_WEEK); break; case "day": long day = toLongWithRange("day", m.group(1), - Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); + Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); result = new CalendarInterval(0, day * MICROS_PER_DAY); break; case "hour": long hour = toLongWithRange("hour", m.group(1), - Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); + Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); result = new CalendarInterval(0, hour * MICROS_PER_HOUR); break; case "minute": long minute = toLongWithRange("minute", m.group(1), - Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); + Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); break; case "second": { @@ -242,7 +241,7 @@ public static CalendarInterval fromSingleUnitString(String unit, String s) } case "millisecond": long millisecond = toLongWithRange("millisecond", m.group(1), - Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); + Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); break; case "microsecond": { @@ -265,17 +264,17 @@ public static long parseSecondNano(String secondNano) throws IllegalArgumentExce String[] parts = secondNano.split("\\."); if (parts.length == 1) { return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND, - Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND; + Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND; } else if (parts.length == 2) { long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0], - Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND); + Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND); long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L); return seconds * MICROS_PER_SECOND + nanos / 1000L; } else { throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn"); + "Interval string does not match second-nano format of ss.nnnnnnnnn"); } } 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 b65e798be62c..de1567c4009f 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,6 +17,7 @@ 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.UTF8String; From 5801bdc5248aac89522b44ccf07b8bef6d737043 Mon Sep 17 00:00:00 2001 From: Priyanka Garg Date: Mon, 1 Jul 2019 14:47:25 +0530 Subject: [PATCH 5/5] SAPRK-24695: To remove extra spaces --- .../org/apache/spark/sql/types/CalendarInterval.java | 6 +++--- .../spark/sql/catalyst/expressions/arithmetic.scala | 2 +- .../expressions/CollectionExpressionsSuite.scala | 12 ++++++------ .../sql/execution/vectorized/ColumnVectorUtils.java | 8 ++++---- 4 files changed, 14 insertions(+), 14 deletions(-) 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 710066a6be6a..787efe70626f 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 @@ -180,7 +180,7 @@ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumen long nanos = toLongWithRange("nanosecond", m.group(8), 0L, 999999999L); result = new CalendarInterval(0, sign * ( days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + - seconds * MICROS_PER_SECOND + nanos / 1000L)); + seconds * MICROS_PER_SECOND + nanos / 1000L)); } catch (Exception e) { throw new IllegalArgumentException( "Error parsing interval day-time string: " + e.getMessage(), e); @@ -190,7 +190,7 @@ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumen } public static CalendarInterval fromSingleUnitString(String unit, String s) - throws IllegalArgumentException { + throws IllegalArgumentException { CalendarInterval result = null; if (s == null) { @@ -356,4 +356,4 @@ private void appendUnit(StringBuilder sb, long value, String unit) { sb.append(' ').append(value).append(' ').append(unit).append('s'); } } -} \ No newline at end of file +} 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 49609b21278c..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 @@ -58,7 +58,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression protected override def nullSafeEval(input: Any): Any = { if (dataType.isInstanceOf[CalendarIntervalType]) { - input.asInstanceOf[CalendarInterval].negate + input.asInstanceOf[CalendarInterval].negate() } else { numeric.negate(input) } 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 dcf284d03aeb..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 @@ -737,7 +737,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours").negate)), + Literal(CalendarInterval.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -746,7 +746,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(CalendarInterval.fromString("interval 12 hours").negate)), + Literal(CalendarInterval.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -764,7 +764,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month").negate)), + Literal(CalendarInterval.fromString("interval 1 month").negate())), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -773,7 +773,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 day").negate)), + Literal(CalendarInterval.fromString("interval 1 month 1 day").negate())), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -819,7 +819,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2022-04-01 00:00:00")), Literal(Timestamp.valueOf("2017-01-01 00:00:00")), - Literal(CalendarInterval.fromYearMonthString("1-5").negate)), + Literal(CalendarInterval.fromYearMonthString("1-5").negate())), Seq( Timestamp.valueOf("2022-04-01 00:00:00.000"), Timestamp.valueOf("2020-11-01 00:00:00.000"), @@ -911,7 +911,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(CalendarInterval.fromString("interval 1 month").negate)), + Literal(CalendarInterval.fromString("interval 1 month").negate())), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * CalendarInterval.MICROS_PER_DAY}") } 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 23aa60258be9..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 @@ -88,8 +88,8 @@ public static void populate(WritableColumnVector col, InternalRow row, int field } } else if (t instanceof CalendarIntervalType) { CalendarInterval c = (CalendarInterval)row.get(fieldIdx, t); - col.getChild(0).putInts(0, capacity, c.months()); - col.getChild(1).putLongs(0, capacity, c.microseconds()); + col.getChild(0).putInts(0, capacity, c.months); + col.getChild(1).putLongs(0, capacity, c.microseconds); } else if (t instanceof DateType) { col.putInts(0, capacity, row.getInt(fieldIdx)); } else if (t instanceof TimestampType) { @@ -164,8 +164,8 @@ private static void appendValue(WritableColumnVector dst, DataType t, Object o) } else if (t instanceof CalendarIntervalType) { CalendarInterval c = (CalendarInterval)o; dst.appendStruct(false); - dst.getChild(0).appendInt(c.months()); - dst.getChild(1).appendLong(c.microseconds()); + dst.getChild(0).appendInt(c.months); + dst.getChild(1).appendLong(c.microseconds); } else if (t instanceof DateType) { dst.appendInt(DateTimeUtils.fromJavaDate((Date)o)); } else {