You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2015/10/23 09:00:28 UTC
[1/2] spark git commit: [SPARK-11273][SQL] Move
ArrayData/MapData/DataTypeParser to catalyst.util package
Repository: spark
Updated Branches:
refs/heads/master b1c1597e3 -> cdea0174e
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
new file mode 100644
index 0000000..5068394
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
@@ -0,0 +1,195 @@
+/*
+ * 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.apache.spark.SparkFunSuite
+import org.scalatest.PrivateMethodTester
+
+import scala.language.postfixOps
+
+class DecimalSuite extends SparkFunSuite with PrivateMethodTester {
+ /** Check that a Decimal has the given string representation, precision and scale */
+ private def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = {
+ assert(d.toString === string)
+ assert(d.precision === precision)
+ assert(d.scale === scale)
+ }
+
+ test("creating decimals") {
+ checkDecimal(new Decimal(), "0", 1, 0)
+ checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3)
+ checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1)
+ checkDecimal(Decimal(BigDecimal("-9.95"), 4, 1), "-10.0", 4, 1)
+ checkDecimal(Decimal("10.030"), "10.030", 5, 3)
+ checkDecimal(Decimal(10.03), "10.03", 4, 2)
+ checkDecimal(Decimal(17L), "17", 20, 0)
+ checkDecimal(Decimal(17), "17", 10, 0)
+ checkDecimal(Decimal(17L, 2, 1), "1.7", 2, 1)
+ checkDecimal(Decimal(170L, 4, 2), "1.70", 4, 2)
+ checkDecimal(Decimal(17L, 24, 1), "1.7", 24, 1)
+ checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0)
+ checkDecimal(Decimal(1000000000000000000L, 20, 2), "10000000000000000.00", 20, 2)
+ checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0)
+ checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0)
+ intercept[IllegalArgumentException](Decimal(170L, 2, 1))
+ intercept[IllegalArgumentException](Decimal(170L, 2, 0))
+ intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1))
+ intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1))
+ intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0))
+ }
+
+ test("creating decimals with negative scale") {
+ checkDecimal(Decimal(BigDecimal("98765"), 5, -3), "9.9E+4", 5, -3)
+ checkDecimal(Decimal(BigDecimal("314.159"), 6, -2), "3E+2", 6, -2)
+ checkDecimal(Decimal(BigDecimal(1.579e12), 4, -9), "1.579E+12", 4, -9)
+ checkDecimal(Decimal(BigDecimal(1.579e12), 4, -10), "1.58E+12", 4, -10)
+ checkDecimal(Decimal(103050709L, 9, -10), "1.03050709E+18", 9, -10)
+ checkDecimal(Decimal(1e8.toLong, 10, -10), "1.00000000E+18", 10, -10)
+ }
+
+ test("double and long values") {
+ /** Check that a Decimal converts to the given double and long values */
+ def checkValues(d: Decimal, doubleValue: Double, longValue: Long): Unit = {
+ assert(d.toDouble === doubleValue)
+ assert(d.toLong === longValue)
+ }
+
+ checkValues(new Decimal(), 0.0, 0L)
+ checkValues(Decimal(BigDecimal("10.030")), 10.03, 10L)
+ checkValues(Decimal(BigDecimal("10.030"), 4, 1), 10.0, 10L)
+ checkValues(Decimal(BigDecimal("-9.95"), 4, 1), -10.0, -10L)
+ checkValues(Decimal(10.03), 10.03, 10L)
+ checkValues(Decimal(17L), 17.0, 17L)
+ checkValues(Decimal(17), 17.0, 17L)
+ checkValues(Decimal(17L, 2, 1), 1.7, 1L)
+ checkValues(Decimal(170L, 4, 2), 1.7, 1L)
+ checkValues(Decimal(1e16.toLong), 1e16, 1e16.toLong)
+ checkValues(Decimal(1e17.toLong), 1e17, 1e17.toLong)
+ checkValues(Decimal(1e18.toLong), 1e18, 1e18.toLong)
+ checkValues(Decimal(2e18.toLong), 2e18, 2e18.toLong)
+ checkValues(Decimal(Long.MaxValue), Long.MaxValue.toDouble, Long.MaxValue)
+ checkValues(Decimal(Long.MinValue), Long.MinValue.toDouble, Long.MinValue)
+ checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L)
+ checkValues(Decimal(Double.MinValue), Double.MinValue, 0L)
+ }
+
+ // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs
+ private val decimalVal = PrivateMethod[BigDecimal]('decimalVal)
+
+ /** Check whether a decimal is represented compactly (passing whether we expect it to be) */
+ private def checkCompact(d: Decimal, expected: Boolean): Unit = {
+ val isCompact = d.invokePrivate(decimalVal()).eq(null)
+ assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact")
+ }
+
+ test("small decimals represented as unscaled long") {
+ checkCompact(new Decimal(), true)
+ checkCompact(Decimal(BigDecimal(10.03)), false)
+ checkCompact(Decimal(BigDecimal(1e20)), false)
+ checkCompact(Decimal(17L), true)
+ checkCompact(Decimal(17), true)
+ checkCompact(Decimal(17L, 2, 1), true)
+ checkCompact(Decimal(170L, 4, 2), true)
+ checkCompact(Decimal(17L, 24, 1), true)
+ checkCompact(Decimal(1e16.toLong), true)
+ checkCompact(Decimal(1e17.toLong), true)
+ checkCompact(Decimal(1e18.toLong - 1), true)
+ checkCompact(Decimal(- 1e18.toLong + 1), true)
+ checkCompact(Decimal(1e18.toLong - 1, 30, 10), true)
+ checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true)
+ checkCompact(Decimal(1e18.toLong), false)
+ checkCompact(Decimal(-1e18.toLong), false)
+ checkCompact(Decimal(1e18.toLong, 30, 10), false)
+ checkCompact(Decimal(-1e18.toLong, 30, 10), false)
+ checkCompact(Decimal(Long.MaxValue), false)
+ checkCompact(Decimal(Long.MinValue), false)
+ }
+
+ test("hash code") {
+ assert(Decimal(123).hashCode() === (123).##)
+ assert(Decimal(-123).hashCode() === (-123).##)
+ assert(Decimal(Int.MaxValue).hashCode() === Int.MaxValue.##)
+ assert(Decimal(Long.MaxValue).hashCode() === Long.MaxValue.##)
+ assert(Decimal(BigDecimal(123)).hashCode() === (123).##)
+
+ val reallyBig = BigDecimal("123182312312313232112312312123.1231231231")
+ assert(Decimal(reallyBig).hashCode() === reallyBig.hashCode)
+ }
+
+ test("equals") {
+ // The decimals on the left are stored compactly, while the ones on the right aren't
+ checkCompact(Decimal(123), true)
+ checkCompact(Decimal(BigDecimal(123)), false)
+ checkCompact(Decimal("123"), false)
+ assert(Decimal(123) === Decimal(BigDecimal(123)))
+ assert(Decimal(123) === Decimal(BigDecimal("123.00")))
+ assert(Decimal(-123) === Decimal(BigDecimal(-123)))
+ assert(Decimal(-123) === Decimal(BigDecimal("-123.00")))
+ }
+
+ test("isZero") {
+ assert(Decimal(0).isZero)
+ assert(Decimal(0, 4, 2).isZero)
+ assert(Decimal("0").isZero)
+ assert(Decimal("0.000").isZero)
+ assert(!Decimal(1).isZero)
+ assert(!Decimal(1, 4, 2).isZero)
+ assert(!Decimal("1").isZero)
+ assert(!Decimal("0.001").isZero)
+ }
+
+ test("arithmetic") {
+ assert(Decimal(100) + Decimal(-100) === Decimal(0))
+ assert(Decimal(100) + Decimal(-100) === Decimal(0))
+ assert(Decimal(100) * Decimal(-100) === Decimal(-10000))
+ assert(Decimal(1e13) * Decimal(1e13) === Decimal(1e26))
+ assert(Decimal(100) / Decimal(-100) === Decimal(-1))
+ assert(Decimal(100) / Decimal(0) === null)
+ assert(Decimal(100) % Decimal(-100) === Decimal(0))
+ assert(Decimal(100) % Decimal(3) === Decimal(1))
+ assert(Decimal(-100) % Decimal(3) === Decimal(-1))
+ assert(Decimal(100) % Decimal(0) === null)
+ }
+
+ // regression test for SPARK-8359
+ test("accurate precision after multiplication") {
+ val decimal = (Decimal(Long.MaxValue, 38, 0) * Decimal(Long.MaxValue, 38, 0)).toJavaBigDecimal
+ assert(decimal.unscaledValue.toString === "85070591730234615847396907784232501249")
+ }
+
+ // regression test for SPARK-8677
+ test("fix non-terminating decimal expansion problem") {
+ val decimal = Decimal(1.0, 10, 3) / Decimal(3.0, 10, 3)
+ // The difference between decimal should not be more than 0.001.
+ assert(decimal.toDouble - 0.333 < 0.001)
+ }
+
+ // regression test for SPARK-8800
+ test("fix loss of precision/scale when doing division operation") {
+ val a = Decimal(2) / Decimal(3)
+ assert(a.toDouble < 1.0 && a.toDouble > 0.6)
+ val b = Decimal(1) / Decimal(8)
+ assert(b.toDouble === 0.125)
+ }
+
+ test("set/setOrNull") {
+ assert(new Decimal().set(10L, 10, 0).toUnscaledLong === 10L)
+ assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L)
+ assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue)
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
deleted file mode 100644
index f9aceb8..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala
+++ /dev/null
@@ -1,196 +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.decimal
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.types.Decimal
-import org.scalatest.PrivateMethodTester
-
-import scala.language.postfixOps
-
-class DecimalSuite extends SparkFunSuite with PrivateMethodTester {
- /** Check that a Decimal has the given string representation, precision and scale */
- private def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = {
- assert(d.toString === string)
- assert(d.precision === precision)
- assert(d.scale === scale)
- }
-
- test("creating decimals") {
- checkDecimal(new Decimal(), "0", 1, 0)
- checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3)
- checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1)
- checkDecimal(Decimal(BigDecimal("-9.95"), 4, 1), "-10.0", 4, 1)
- checkDecimal(Decimal("10.030"), "10.030", 5, 3)
- checkDecimal(Decimal(10.03), "10.03", 4, 2)
- checkDecimal(Decimal(17L), "17", 20, 0)
- checkDecimal(Decimal(17), "17", 10, 0)
- checkDecimal(Decimal(17L, 2, 1), "1.7", 2, 1)
- checkDecimal(Decimal(170L, 4, 2), "1.70", 4, 2)
- checkDecimal(Decimal(17L, 24, 1), "1.7", 24, 1)
- checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0)
- checkDecimal(Decimal(1000000000000000000L, 20, 2), "10000000000000000.00", 20, 2)
- checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0)
- checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0)
- intercept[IllegalArgumentException](Decimal(170L, 2, 1))
- intercept[IllegalArgumentException](Decimal(170L, 2, 0))
- intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1))
- intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1))
- intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0))
- }
-
- test("creating decimals with negative scale") {
- checkDecimal(Decimal(BigDecimal("98765"), 5, -3), "9.9E+4", 5, -3)
- checkDecimal(Decimal(BigDecimal("314.159"), 6, -2), "3E+2", 6, -2)
- checkDecimal(Decimal(BigDecimal(1.579e12), 4, -9), "1.579E+12", 4, -9)
- checkDecimal(Decimal(BigDecimal(1.579e12), 4, -10), "1.58E+12", 4, -10)
- checkDecimal(Decimal(103050709L, 9, -10), "1.03050709E+18", 9, -10)
- checkDecimal(Decimal(1e8.toLong, 10, -10), "1.00000000E+18", 10, -10)
- }
-
- test("double and long values") {
- /** Check that a Decimal converts to the given double and long values */
- def checkValues(d: Decimal, doubleValue: Double, longValue: Long): Unit = {
- assert(d.toDouble === doubleValue)
- assert(d.toLong === longValue)
- }
-
- checkValues(new Decimal(), 0.0, 0L)
- checkValues(Decimal(BigDecimal("10.030")), 10.03, 10L)
- checkValues(Decimal(BigDecimal("10.030"), 4, 1), 10.0, 10L)
- checkValues(Decimal(BigDecimal("-9.95"), 4, 1), -10.0, -10L)
- checkValues(Decimal(10.03), 10.03, 10L)
- checkValues(Decimal(17L), 17.0, 17L)
- checkValues(Decimal(17), 17.0, 17L)
- checkValues(Decimal(17L, 2, 1), 1.7, 1L)
- checkValues(Decimal(170L, 4, 2), 1.7, 1L)
- checkValues(Decimal(1e16.toLong), 1e16, 1e16.toLong)
- checkValues(Decimal(1e17.toLong), 1e17, 1e17.toLong)
- checkValues(Decimal(1e18.toLong), 1e18, 1e18.toLong)
- checkValues(Decimal(2e18.toLong), 2e18, 2e18.toLong)
- checkValues(Decimal(Long.MaxValue), Long.MaxValue.toDouble, Long.MaxValue)
- checkValues(Decimal(Long.MinValue), Long.MinValue.toDouble, Long.MinValue)
- checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L)
- checkValues(Decimal(Double.MinValue), Double.MinValue, 0L)
- }
-
- // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs
- private val decimalVal = PrivateMethod[BigDecimal]('decimalVal)
-
- /** Check whether a decimal is represented compactly (passing whether we expect it to be) */
- private def checkCompact(d: Decimal, expected: Boolean): Unit = {
- val isCompact = d.invokePrivate(decimalVal()).eq(null)
- assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact")
- }
-
- test("small decimals represented as unscaled long") {
- checkCompact(new Decimal(), true)
- checkCompact(Decimal(BigDecimal(10.03)), false)
- checkCompact(Decimal(BigDecimal(1e20)), false)
- checkCompact(Decimal(17L), true)
- checkCompact(Decimal(17), true)
- checkCompact(Decimal(17L, 2, 1), true)
- checkCompact(Decimal(170L, 4, 2), true)
- checkCompact(Decimal(17L, 24, 1), true)
- checkCompact(Decimal(1e16.toLong), true)
- checkCompact(Decimal(1e17.toLong), true)
- checkCompact(Decimal(1e18.toLong - 1), true)
- checkCompact(Decimal(- 1e18.toLong + 1), true)
- checkCompact(Decimal(1e18.toLong - 1, 30, 10), true)
- checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true)
- checkCompact(Decimal(1e18.toLong), false)
- checkCompact(Decimal(-1e18.toLong), false)
- checkCompact(Decimal(1e18.toLong, 30, 10), false)
- checkCompact(Decimal(-1e18.toLong, 30, 10), false)
- checkCompact(Decimal(Long.MaxValue), false)
- checkCompact(Decimal(Long.MinValue), false)
- }
-
- test("hash code") {
- assert(Decimal(123).hashCode() === (123).##)
- assert(Decimal(-123).hashCode() === (-123).##)
- assert(Decimal(Int.MaxValue).hashCode() === Int.MaxValue.##)
- assert(Decimal(Long.MaxValue).hashCode() === Long.MaxValue.##)
- assert(Decimal(BigDecimal(123)).hashCode() === (123).##)
-
- val reallyBig = BigDecimal("123182312312313232112312312123.1231231231")
- assert(Decimal(reallyBig).hashCode() === reallyBig.hashCode)
- }
-
- test("equals") {
- // The decimals on the left are stored compactly, while the ones on the right aren't
- checkCompact(Decimal(123), true)
- checkCompact(Decimal(BigDecimal(123)), false)
- checkCompact(Decimal("123"), false)
- assert(Decimal(123) === Decimal(BigDecimal(123)))
- assert(Decimal(123) === Decimal(BigDecimal("123.00")))
- assert(Decimal(-123) === Decimal(BigDecimal(-123)))
- assert(Decimal(-123) === Decimal(BigDecimal("-123.00")))
- }
-
- test("isZero") {
- assert(Decimal(0).isZero)
- assert(Decimal(0, 4, 2).isZero)
- assert(Decimal("0").isZero)
- assert(Decimal("0.000").isZero)
- assert(!Decimal(1).isZero)
- assert(!Decimal(1, 4, 2).isZero)
- assert(!Decimal("1").isZero)
- assert(!Decimal("0.001").isZero)
- }
-
- test("arithmetic") {
- assert(Decimal(100) + Decimal(-100) === Decimal(0))
- assert(Decimal(100) + Decimal(-100) === Decimal(0))
- assert(Decimal(100) * Decimal(-100) === Decimal(-10000))
- assert(Decimal(1e13) * Decimal(1e13) === Decimal(1e26))
- assert(Decimal(100) / Decimal(-100) === Decimal(-1))
- assert(Decimal(100) / Decimal(0) === null)
- assert(Decimal(100) % Decimal(-100) === Decimal(0))
- assert(Decimal(100) % Decimal(3) === Decimal(1))
- assert(Decimal(-100) % Decimal(3) === Decimal(-1))
- assert(Decimal(100) % Decimal(0) === null)
- }
-
- // regression test for SPARK-8359
- test("accurate precision after multiplication") {
- val decimal = (Decimal(Long.MaxValue, 38, 0) * Decimal(Long.MaxValue, 38, 0)).toJavaBigDecimal
- assert(decimal.unscaledValue.toString === "85070591730234615847396907784232501249")
- }
-
- // regression test for SPARK-8677
- test("fix non-terminating decimal expansion problem") {
- val decimal = Decimal(1.0, 10, 3) / Decimal(3.0, 10, 3)
- // The difference between decimal should not be more than 0.001.
- assert(decimal.toDouble - 0.333 < 0.001)
- }
-
- // regression test for SPARK-8800
- test("fix loss of precision/scale when doing division operation") {
- val a = Decimal(2) / Decimal(3)
- assert(a.toDouble < 1.0 && a.toDouble > 0.6)
- val b = Decimal(1) / Decimal(8)
- assert(b.toDouble === 0.125)
- }
-
- test("set/setOrNull") {
- assert(new Decimal().set(10L, 10, 0).toUnscaledLong === 10L)
- assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L)
- assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue)
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
index de11a16..e4f4cf1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
@@ -17,15 +17,15 @@
package org.apache.spark.sql
-
import scala.language.implicitConversions
import org.apache.spark.annotation.Experimental
import org.apache.spark.Logging
import org.apache.spark.sql.functions.lit
+import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.encoders.Encoder
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
index 446739d..6969b42 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala
@@ -25,6 +25,7 @@ import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.{TableIdentifier, AbstractSparkSQLParser}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
index d7d6ede..3f34520 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.datasources.json
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.{MapData, ArrayData, DateTimeUtils}
import scala.collection.Map
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala
index 09b8a9e..b2e5201 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala
@@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.datasources.json.JacksonUtils.nextUntil
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala
index 49007e4..b16c465 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala
@@ -32,7 +32,7 @@ import org.apache.parquet.schema.{GroupType, MessageType, PrimitiveType, Type}
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData, DateTimeUtils}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala
index d4e6980..d611b00 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala
@@ -24,6 +24,7 @@ import scala.collection.JavaConverters._
import net.razorvine.pickle._
+import org.apache.spark.{Logging => SparkLogging, TaskContext, Accumulator}
import org.apache.spark.api.python.{PythonRunner, PythonBroadcast, PythonRDD, SerDeUtil}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
@@ -33,9 +34,9 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.{MapData, GenericArrayData, ArrayBasedMapData, ArrayData}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.{Logging => SparkLogging, TaskContext, Accumulator}
/**
* A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]].
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
index a741a45..8d4854b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.test
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData}
import org.apache.spark.sql.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
index 7d1ee39..00f1526 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala
@@ -20,9 +20,10 @@ package org.apache.spark.sql
import java.io.ByteArrayOutputStream
import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.serializer.{KryoSerializer, JavaSerializer}
+import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.memory.MemoryAllocator
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index d17671d..a229e58 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData}
+
import scala.beans.{BeanInfo, BeanProperty}
import com.clearspring.analytics.stream.cardinality.HyperLogLog
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
index 964cdb5..a5882f7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
@@ -22,7 +22,8 @@ import scala.util.Random
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericMutableRow}
-import org.apache.spark.sql.types.{ArrayBasedMapData, GenericArrayData, AtomicType, Decimal}
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData}
+import org.apache.spark.sql.types.{AtomicType, Decimal}
import org.apache.spark.unsafe.types.UTF8String
object ColumnarTestUtils {
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
index 5dc37e5..b3fceea 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
@@ -21,8 +21,9 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Literal, IsNull}
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.sql.types.{GenericArrayData, ArrayType, StringType}
+import org.apache.spark.sql.types.{ArrayType, StringType}
import org.apache.spark.unsafe.types.UTF8String
class RowFormatConvertersSuite extends SparkPlanTest with SharedSQLContext {
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 43c238f..36f0708 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -28,7 +28,7 @@ import org.apache.hadoop.{io => hadoopIo}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.types._
import org.apache.spark.sql.{AnalysisException, types}
import org.apache.spark.unsafe.types.UTF8String
@@ -50,8 +50,8 @@ import org.apache.spark.unsafe.types.UTF8String
* java.sql.Date
* java.sql.Timestamp
* Complex Types =>
- * Map: [[org.apache.spark.sql.types.MapData]]
- * List: [[org.apache.spark.sql.types.ArrayData]]
+ * Map: [[MapData]]
+ * List: [[ArrayData]]
* Struct: [[org.apache.spark.sql.catalyst.InternalRow]]
* Union: NOT SUPPORTED YET
* The Complex types plays as a container, which can hold arbitrary data types.
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 5819cb9..fdb576b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -32,11 +32,12 @@ import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog}
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.util.DataTypeParser
import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource}
import org.apache.spark.sql.execution.{FileRelation, datasources}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
index f57b206..2ccad47 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.ArrayData
import org.apache.spark.sql.hive.HiveShim._
import org.apache.spark.sql.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
index 81a70b8..8bb9058 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
@@ -30,6 +30,7 @@ import org.apache.hadoop.io.LongWritable
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.util.{MapData, GenericArrayData, ArrayBasedMapData}
import org.apache.spark.sql.types._
import org.apache.spark.sql.Row
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org
[2/2] spark git commit: [SPARK-11273][SQL] Move
ArrayData/MapData/DataTypeParser to catalyst.util package
Posted by rx...@apache.org.
[SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.util package
Author: Reynold Xin <rx...@databricks.com>
Closes #9239 from rxin/types-private.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cdea0174
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cdea0174
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cdea0174
Branch: refs/heads/master
Commit: cdea0174e32a5f4c28fd59899b2e9774994303d5
Parents: b1c1597
Author: Reynold Xin <rx...@databricks.com>
Authored: Fri Oct 23 00:00:21 2015 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Fri Oct 23 00:00:21 2015 -0700
----------------------------------------------------------------------
.../apache/spark/mllib/linalg/Matrices.scala | 1 +
.../org/apache/spark/mllib/linalg/Vectors.scala | 1 +
.../expressions/SpecializedGetters.java | 4 +-
.../catalyst/expressions/UnsafeArrayData.java | 1 +
.../sql/catalyst/expressions/UnsafeMapData.java | 2 +-
.../sql/execution/UnsafeExternalRowSorter.java | 2 +-
.../spark/sql/AbstractScalaRowIterator.scala | 30 ---
.../sql/catalyst/CatalystTypeConverters.scala | 2 +-
.../spark/sql/catalyst/ScalaReflection.scala | 2 +-
.../apache/spark/sql/catalyst/SqlParser.scala | 1 +
.../sql/catalyst/encoders/RowEncoder.scala | 2 +-
.../spark/sql/catalyst/expressions/Cast.scala | 2 +-
.../sql/catalyst/expressions/JoinedRow.scala | 1 +
.../sql/catalyst/expressions/aggregates.scala | 2 +-
.../expressions/codegen/CodeGenerator.scala | 1 +
.../codegen/GenerateSafeProjection.scala | 1 +
.../expressions/collectionOperations.scala | 1 +
.../expressions/complexTypeCreator.scala | 2 +-
.../expressions/complexTypeExtractors.scala | 1 +
.../sql/catalyst/expressions/generators.scala | 1 +
.../sql/catalyst/expressions/objects.scala | 1 +
.../expressions/regexpExpressions.scala | 2 +-
.../spark/sql/catalyst/expressions/rows.scala | 1 +
.../expressions/stringExpressions.scala | 1 +
.../util/AbstractScalaRowIterator.scala | 30 +++
.../sql/catalyst/util/ArrayBasedMapData.scala | 73 +++++++
.../spark/sql/catalyst/util/ArrayData.scala | 140 +++++++++++++
.../sql/catalyst/util/DataTypeParser.scala | 116 +++++++++++
.../sql/catalyst/util/GenericArrayData.scala | 136 +++++++++++++
.../spark/sql/catalyst/util/MapData.scala | 42 ++++
.../spark/sql/types/ArrayBasedMapData.scala | 73 -------
.../org/apache/spark/sql/types/ArrayData.scala | 139 -------------
.../apache/spark/sql/types/DataTypeParser.scala | 115 -----------
.../spark/sql/types/GenericArrayData.scala | 135 -------------
.../org/apache/spark/sql/types/MapData.scala | 40 ----
.../org/apache/spark/sql/types/StructType.scala | 2 +-
.../catalyst/encoders/ProductEncoderSuite.scala | 20 +-
.../expressions/UnsafeRowConverterSuite.scala | 2 +-
.../codegen/GeneratedProjectionSuite.scala | 1 +
.../sql/catalyst/util/DataTypeParserSuite.scala | 119 +++++++++++
.../spark/sql/types/DataTypeParserSuite.scala | 118 -----------
.../apache/spark/sql/types/DecimalSuite.scala | 195 ++++++++++++++++++
.../spark/sql/types/decimal/DecimalSuite.scala | 196 -------------------
.../scala/org/apache/spark/sql/Column.scala | 4 +-
.../sql/execution/datasources/DDLParser.scala | 1 +
.../datasources/json/JacksonGenerator.scala | 2 +-
.../datasources/json/JacksonParser.scala | 2 +-
.../parquet/CatalystRowConverter.scala | 2 +-
.../org/apache/spark/sql/execution/python.scala | 3 +-
.../apache/spark/sql/test/ExamplePointUDT.scala | 1 +
.../org/apache/spark/sql/UnsafeRowSuite.scala | 5 +-
.../apache/spark/sql/UserDefinedTypeSuite.scala | 2 +
.../spark/sql/columnar/ColumnarTestUtils.scala | 3 +-
.../execution/RowFormatConvertersSuite.scala | 3 +-
.../apache/spark/sql/hive/HiveInspectors.scala | 6 +-
.../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +-
.../org/apache/spark/sql/hive/hiveUDFs.scala | 1 +
.../spark/sql/hive/HiveInspectorSuite.scala | 1 +
58 files changed, 912 insertions(+), 884 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
index 8ba6e4e..8879dcf 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
@@ -26,6 +26,7 @@ import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM}
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
index 3642e92..dcdc614 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -30,6 +30,7 @@ import org.apache.spark.annotation.{AlphaComponent, Since}
import org.apache.spark.mllib.util.NumericParser
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java
----------------------------------------------------------------------
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 8f1027f..eea7149 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
@@ -18,10 +18,10 @@
package org.apache.spark.sql.catalyst.expressions;
import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.types.ArrayData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.types.MapData;
+import org.apache.spark.sql.catalyst.util.MapData;
import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
----------------------------------------------------------------------
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 761f044..3513960 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
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
+import org.apache.spark.sql.catalyst.util.ArrayData;
import org.apache.spark.sql.types.*;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java
index 5bebe2a..651eb1f 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions;
import java.nio.ByteBuffer;
-import org.apache.spark.sql.types.MapData;
+import org.apache.spark.sql.catalyst.util.MapData;
import org.apache.spark.unsafe.Platform;
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
index 1d27182..7d94e05 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
@@ -26,7 +26,7 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.spark.SparkEnv;
import org.apache.spark.TaskContext;
-import org.apache.spark.sql.AbstractScalaRowIterator;
+import org.apache.spark.sql.catalyst.util.AbstractScalaRowIterator;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.UnsafeProjection;
import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/AbstractScalaRowIterator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AbstractScalaRowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AbstractScalaRowIterator.scala
deleted file mode 100644
index 1090bdb..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AbstractScalaRowIterator.scala
+++ /dev/null
@@ -1,30 +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
-
-/**
- * Shim to allow us to implement [[scala.Iterator]] in Java. Scala 2.11+ has an AbstractIterator
- * class for this, but that class is `private[scala]` in 2.10. We need to explicitly fix this to
- * `Row` in order to work around a spurious IntelliJ compiler error. This cannot be an abstract
- * class because that leads to compilation errors under Scala 2.11.
- */
-private[spark] class AbstractScalaRowIterator[T] extends Iterator[T] {
- override def hasNext: Boolean = throw new NotImplementedError
-
- override def next(): T = throw new NotImplementedError
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
index f255917..2ec0ff5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
@@ -27,7 +27,7 @@ import scala.language.existentials
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
----------------------------------------------------------------------
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 713c6b5..c25161e 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
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.analysis.{UnresolvedExtractValue, UnresolvedAttribute}
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData, ArrayData, DateTimeUtils}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 08ca325..833368b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis._
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.util.DataTypeParser
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 5142856..e9cc00a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -22,7 +22,7 @@ import scala.reflect.ClassTag
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData, DateTimeUtils}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
----------------------------------------------------------------------
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 99d7444..5564e24 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
@@ -22,7 +22,7 @@ import java.math.{BigDecimal => JavaBigDecimal}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen._
-import org.apache.spark.sql.catalyst.util.{StringUtils, DateTimeUtils}
+import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala
----------------------------------------------------------------------
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 d3560df..935c3aa 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
@@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.{MapData, ArrayData}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index 95061c4..70819be 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
-import org.apache.spark.sql.catalyst.util.TypeUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData, TypeUtils}
import org.apache.spark.sql.types._
import org.apache.spark.util.collection.OpenHashSet
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
----------------------------------------------------------------------
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 a4ec508..f0f7a6c 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
@@ -27,6 +27,7 @@ import org.codehaus.janino.ClassBodyEvaluator
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.util.{MapData, ArrayData}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
index ee50587..f0ed864 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData}
import org.apache.spark.sql.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
----------------------------------------------------------------------
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 75c66bc..89d8772 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
@@ -20,6 +20,7 @@ import java.util.Comparator
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, CodegenFallback, GeneratedExpressionCode}
+import org.apache.spark.sql.catalyst.util.{MapData, GenericArrayData, ArrayData}
import org.apache.spark.sql.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 059e45b..1854dfa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -21,7 +21,7 @@ import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen._
-import org.apache.spark.sql.catalyst.util.TypeUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, TypeUtils}
import org.apache.spark.sql.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index a2b5a6a..41cd0a1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -21,6 +21,7 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
+import org.apache.spark.sql.catalyst.util.{MapData, GenericArrayData, ArrayData}
import org.apache.spark.sql.types._
////////////////////////////////////////////////////////////////////////////////////////////////////
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index c0845e1..1a2092c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.util.{MapData, ArrayData}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
index b42d6c5..8185528 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer
import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation}
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import scala.language.existentials
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 64f1594..9e484c5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -22,7 +22,7 @@ import java.util.regex.{MatchResult, Pattern}
import org.apache.commons.lang3.StringEscapeUtils
import org.apache.spark.sql.catalyst.expressions.codegen._
-import org.apache.spark.sql.catalyst.util.StringUtils
+import org.apache.spark.sql.catalyst.util.{GenericArrayData, StringUtils}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
----------------------------------------------------------------------
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 017efd2..cfc68fc 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.{MapData, ArrayData}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index abc5c94..8770c4b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -22,6 +22,7 @@ import java.util.{HashMap, Locale, Map => JMap}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.util.ArrayData
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{ByteArray, UTF8String}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
new file mode 100644
index 0000000..6d35f14
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.catalyst.util
+
+/**
+ * Shim to allow us to implement [[scala.Iterator]] in Java. Scala 2.11+ has an AbstractIterator
+ * class for this, but that class is `private[scala]` in 2.10. We need to explicitly fix this to
+ * `Row` in order to work around a spurious IntelliJ compiler error. This cannot be an abstract
+ * class because that leads to compilation errors under Scala 2.11.
+ */
+private[spark] class AbstractScalaRowIterator[T] extends Iterator[T] {
+ override def hasNext: Boolean = throw new NotImplementedError
+
+ override def next(): T = throw new NotImplementedError
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala
new file mode 100644
index 0000000..70b028d
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.catalyst.util
+
+class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData {
+ require(keyArray.numElements() == valueArray.numElements())
+
+ override def numElements(): Int = keyArray.numElements()
+
+ override def copy(): MapData = new ArrayBasedMapData(keyArray.copy(), valueArray.copy())
+
+ // We need to check equality of map type in tests.
+ override def equals(o: Any): Boolean = {
+ if (!o.isInstanceOf[ArrayBasedMapData]) {
+ return false
+ }
+
+ val other = o.asInstanceOf[ArrayBasedMapData]
+ if (other eq null) {
+ return false
+ }
+
+ ArrayBasedMapData.toScalaMap(this) == ArrayBasedMapData.toScalaMap(other)
+ }
+
+ override def hashCode: Int = {
+ ArrayBasedMapData.toScalaMap(this).hashCode()
+ }
+
+ override def toString: String = {
+ s"keys: $keyArray, values: $valueArray"
+ }
+}
+
+object ArrayBasedMapData {
+ def apply(map: Map[Any, Any]): ArrayBasedMapData = {
+ val array = map.toArray
+ ArrayBasedMapData(array.map(_._1), array.map(_._2))
+ }
+
+ def apply(keys: Array[Any], values: Array[Any]): ArrayBasedMapData = {
+ new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values))
+ }
+
+ def toScalaMap(map: ArrayBasedMapData): Map[Any, Any] = {
+ val keys = map.keyArray.asInstanceOf[GenericArrayData].array
+ val values = map.valueArray.asInstanceOf[GenericArrayData].array
+ keys.zip(values).toMap
+ }
+
+ def toScalaMap(keys: Array[Any], values: Array[Any]): Map[Any, Any] = {
+ keys.zip(values).toMap
+ }
+
+ def toScalaMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = {
+ keys.zip(values).toMap
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala
new file mode 100644
index 0000000..cad4a08
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala
@@ -0,0 +1,140 @@
+/*
+ * 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.catalyst.util
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.SpecializedGetters
+import org.apache.spark.sql.types.DataType
+
+abstract class ArrayData extends SpecializedGetters with Serializable {
+ def numElements(): Int
+
+ def copy(): ArrayData
+
+ def array: Array[Any]
+
+ def toBooleanArray(): Array[Boolean] = {
+ val size = numElements()
+ val values = new Array[Boolean](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getBoolean(i)
+ i += 1
+ }
+ values
+ }
+
+ def toByteArray(): Array[Byte] = {
+ val size = numElements()
+ val values = new Array[Byte](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getByte(i)
+ i += 1
+ }
+ values
+ }
+
+ def toShortArray(): Array[Short] = {
+ val size = numElements()
+ val values = new Array[Short](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getShort(i)
+ i += 1
+ }
+ values
+ }
+
+ def toIntArray(): Array[Int] = {
+ val size = numElements()
+ val values = new Array[Int](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getInt(i)
+ i += 1
+ }
+ values
+ }
+
+ def toLongArray(): Array[Long] = {
+ val size = numElements()
+ val values = new Array[Long](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getLong(i)
+ i += 1
+ }
+ values
+ }
+
+ def toFloatArray(): Array[Float] = {
+ val size = numElements()
+ val values = new Array[Float](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getFloat(i)
+ i += 1
+ }
+ values
+ }
+
+ def toDoubleArray(): Array[Double] = {
+ val size = numElements()
+ val values = new Array[Double](size)
+ var i = 0
+ while (i < size) {
+ values(i) = getDouble(i)
+ i += 1
+ }
+ values
+ }
+
+ def toObjectArray(elementType: DataType): Array[AnyRef] =
+ toArray[AnyRef](elementType: DataType)
+
+ def toArray[T: ClassTag](elementType: DataType): Array[T] = {
+ val size = numElements()
+ val values = new Array[T](size)
+ var i = 0
+ while (i < size) {
+ if (isNullAt(i)) {
+ values(i) = null.asInstanceOf[T]
+ } else {
+ values(i) = get(i, elementType).asInstanceOf[T]
+ }
+ i += 1
+ }
+ values
+ }
+
+ // todo: specialize this.
+ def foreach(elementType: DataType, f: (Int, Any) => Unit): Unit = {
+ val size = numElements()
+ var i = 0
+ while (i < size) {
+ if (isNullAt(i)) {
+ f(i, null)
+ } else {
+ f(i, get(i, elementType))
+ }
+ i += 1
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeParser.scala
new file mode 100644
index 0000000..2b83651
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DataTypeParser.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.catalyst.util
+
+import scala.language.implicitConversions
+import scala.util.matching.Regex
+import scala.util.parsing.combinator.syntactical.StandardTokenParsers
+
+import org.apache.spark.sql.catalyst.SqlLexical
+import org.apache.spark.sql.types._
+
+/**
+ * This is a data type parser that can be used to parse string representations of data types
+ * provided in SQL queries. This parser is mixed in with DDLParser and SqlParser.
+ */
+private[sql] trait DataTypeParser extends StandardTokenParsers {
+
+ // This is used to create a parser from a regex. We are using regexes for data type strings
+ // since these strings can be also used as column names or field names.
+ import lexical.Identifier
+ implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch(
+ s"identifier matching regex ${regex}",
+ { case Identifier(str) if regex.unapplySeq(str).isDefined => str }
+ )
+
+ protected lazy val primitiveType: Parser[DataType] =
+ "(?i)string".r ^^^ StringType |
+ "(?i)float".r ^^^ FloatType |
+ "(?i)(?:int|integer)".r ^^^ IntegerType |
+ "(?i)tinyint".r ^^^ ByteType |
+ "(?i)smallint".r ^^^ ShortType |
+ "(?i)double".r ^^^ DoubleType |
+ "(?i)(?:bigint|long)".r ^^^ LongType |
+ "(?i)binary".r ^^^ BinaryType |
+ "(?i)boolean".r ^^^ BooleanType |
+ fixedDecimalType |
+ "(?i)decimal".r ^^^ DecimalType.USER_DEFAULT |
+ "(?i)date".r ^^^ DateType |
+ "(?i)timestamp".r ^^^ TimestampType |
+ varchar
+
+ protected lazy val fixedDecimalType: Parser[DataType] =
+ ("(?i)decimal".r ~> "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ {
+ case precision ~ scale =>
+ DecimalType(precision.toInt, scale.toInt)
+ }
+
+ protected lazy val varchar: Parser[DataType] =
+ "(?i)varchar".r ~> "(" ~> (numericLit <~ ")") ^^^ StringType
+
+ protected lazy val arrayType: Parser[DataType] =
+ "(?i)array".r ~> "<" ~> dataType <~ ">" ^^ {
+ case tpe => ArrayType(tpe)
+ }
+
+ protected lazy val mapType: Parser[DataType] =
+ "(?i)map".r ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ {
+ case t1 ~ _ ~ t2 => MapType(t1, t2)
+ }
+
+ protected lazy val structField: Parser[StructField] =
+ ident ~ ":" ~ dataType ^^ {
+ case name ~ _ ~ tpe => StructField(name, tpe, nullable = true)
+ }
+
+ protected lazy val structType: Parser[DataType] =
+ ("(?i)struct".r ~> "<" ~> repsep(structField, ",") <~ ">" ^^ {
+ case fields => new StructType(fields.toArray)
+ }) |
+ ("(?i)struct".r ~ "<>" ^^^ StructType(Nil))
+
+ protected lazy val dataType: Parser[DataType] =
+ arrayType |
+ mapType |
+ structType |
+ primitiveType
+
+ def toDataType(dataTypeString: String): DataType = synchronized {
+ phrase(dataType)(new lexical.Scanner(dataTypeString)) match {
+ case Success(result, _) => result
+ case failure: NoSuccess => throw new DataTypeException(failMessage(dataTypeString))
+ }
+ }
+
+ private def failMessage(dataTypeString: String): String = {
+ s"Unsupported dataType: $dataTypeString. If you have a struct and a field name of it has " +
+ "any special characters, please use backticks (`) to quote that field name, e.g. `x+y`. " +
+ "Please note that backtick itself is not supported in a field name."
+ }
+}
+
+private[sql] object DataTypeParser {
+ lazy val dataTypeParser = new DataTypeParser {
+ override val lexical = new SqlLexical
+ }
+
+ def parse(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString)
+}
+
+/** The exception thrown from the [[DataTypeParser]]. */
+private[sql] class DataTypeException(message: String) extends Exception(message)
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e9bf7b3
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types.{DataType, Decimal}
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+
+class GenericArrayData(val array: Array[Any]) extends ArrayData {
+
+ def this(seq: scala.collection.GenIterable[Any]) = this(seq.toArray)
+
+ // TODO: This is boxing. We should specialize.
+ def this(primitiveArray: Array[Int]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Long]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Float]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Double]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Short]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq)
+ def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq)
+
+ override def copy(): ArrayData = new GenericArrayData(array.clone())
+
+ override def numElements(): Int = array.length
+
+ private def getAs[T](ordinal: Int) = array(ordinal).asInstanceOf[T]
+ override def isNullAt(ordinal: Int): Boolean = getAs[AnyRef](ordinal) eq null
+ override def get(ordinal: Int, elementType: DataType): AnyRef = getAs(ordinal)
+ override def getBoolean(ordinal: Int): Boolean = getAs(ordinal)
+ override def getByte(ordinal: Int): Byte = getAs(ordinal)
+ override def getShort(ordinal: Int): Short = getAs(ordinal)
+ override def getInt(ordinal: Int): Int = getAs(ordinal)
+ override def getLong(ordinal: Int): Long = getAs(ordinal)
+ override def getFloat(ordinal: Int): Float = getAs(ordinal)
+ override def getDouble(ordinal: Int): Double = getAs(ordinal)
+ override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = getAs(ordinal)
+ override def getUTF8String(ordinal: Int): UTF8String = getAs(ordinal)
+ override def getBinary(ordinal: Int): Array[Byte] = getAs(ordinal)
+ override def getInterval(ordinal: Int): CalendarInterval = getAs(ordinal)
+ override def getStruct(ordinal: Int, numFields: Int): InternalRow = getAs(ordinal)
+ override def getArray(ordinal: Int): ArrayData = getAs(ordinal)
+ override def getMap(ordinal: Int): MapData = getAs(ordinal)
+
+ override def toString(): String = array.mkString("[", ",", "]")
+
+ override def equals(o: Any): Boolean = {
+ if (!o.isInstanceOf[GenericArrayData]) {
+ return false
+ }
+
+ val other = o.asInstanceOf[GenericArrayData]
+ if (other eq null) {
+ return false
+ }
+
+ val len = numElements()
+ if (len != other.numElements()) {
+ return false
+ }
+
+ var i = 0
+ while (i < len) {
+ if (isNullAt(i) != other.isNullAt(i)) {
+ return false
+ }
+ if (!isNullAt(i)) {
+ val o1 = array(i)
+ val o2 = other.array(i)
+ o1 match {
+ case b1: Array[Byte] =>
+ if (!o2.isInstanceOf[Array[Byte]] ||
+ !java.util.Arrays.equals(b1, o2.asInstanceOf[Array[Byte]])) {
+ return false
+ }
+ case f1: Float if java.lang.Float.isNaN(f1) =>
+ if (!o2.isInstanceOf[Float] || ! java.lang.Float.isNaN(o2.asInstanceOf[Float])) {
+ return false
+ }
+ case d1: Double if java.lang.Double.isNaN(d1) =>
+ if (!o2.isInstanceOf[Double] || ! java.lang.Double.isNaN(o2.asInstanceOf[Double])) {
+ return false
+ }
+ case _ => if (o1 != o2) {
+ return false
+ }
+ }
+ }
+ i += 1
+ }
+ true
+ }
+
+ override def hashCode: Int = {
+ var result: Int = 37
+ var i = 0
+ val len = numElements()
+ while (i < len) {
+ val update: Int =
+ if (isNullAt(i)) {
+ 0
+ } else {
+ array(i) match {
+ case b: Boolean => if (b) 0 else 1
+ case b: Byte => b.toInt
+ case s: Short => s.toInt
+ case i: Int => i
+ case l: Long => (l ^ (l >>> 32)).toInt
+ case f: Float => java.lang.Float.floatToIntBits(f)
+ case d: Double =>
+ val b = java.lang.Double.doubleToLongBits(d)
+ (b ^ (b >>> 32)).toInt
+ case a: Array[Byte] => java.util.Arrays.hashCode(a)
+ case other => other.hashCode()
+ }
+ }
+ result = 37 * result + update
+ i += 1
+ }
+ result
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala
new file mode 100644
index 0000000..40db606
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MapData.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.catalyst.util
+
+import org.apache.spark.sql.types.DataType
+
+abstract class MapData extends Serializable {
+
+ def numElements(): Int
+
+ def keyArray(): ArrayData
+
+ def valueArray(): ArrayData
+
+ def copy(): MapData
+
+ def foreach(keyType: DataType, valueType: DataType, f: (Any, Any) => Unit): Unit = {
+ val length = numElements()
+ val keys = keyArray()
+ val values = valueArray()
+ var i = 0
+ while (i < length) {
+ f(keys.get(i, keyType), values.get(i, valueType))
+ i += 1
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayBasedMapData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayBasedMapData.scala
deleted file mode 100644
index e5ffe32..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayBasedMapData.scala
+++ /dev/null
@@ -1,73 +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
-
-class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData {
- require(keyArray.numElements() == valueArray.numElements())
-
- override def numElements(): Int = keyArray.numElements()
-
- override def copy(): MapData = new ArrayBasedMapData(keyArray.copy(), valueArray.copy())
-
- // We need to check equality of map type in tests.
- override def equals(o: Any): Boolean = {
- if (!o.isInstanceOf[ArrayBasedMapData]) {
- return false
- }
-
- val other = o.asInstanceOf[ArrayBasedMapData]
- if (other eq null) {
- return false
- }
-
- ArrayBasedMapData.toScalaMap(this) == ArrayBasedMapData.toScalaMap(other)
- }
-
- override def hashCode: Int = {
- ArrayBasedMapData.toScalaMap(this).hashCode()
- }
-
- override def toString(): String = {
- s"keys: $keyArray, values: $valueArray"
- }
-}
-
-object ArrayBasedMapData {
- def apply(map: Map[Any, Any]): ArrayBasedMapData = {
- val array = map.toArray
- ArrayBasedMapData(array.map(_._1), array.map(_._2))
- }
-
- def apply(keys: Array[Any], values: Array[Any]): ArrayBasedMapData = {
- new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values))
- }
-
- def toScalaMap(map: ArrayBasedMapData): Map[Any, Any] = {
- val keys = map.keyArray.asInstanceOf[GenericArrayData].array
- val values = map.valueArray.asInstanceOf[GenericArrayData].array
- keys.zip(values).toMap
- }
-
- def toScalaMap(keys: Array[Any], values: Array[Any]): Map[Any, Any] = {
- keys.zip(values).toMap
- }
-
- def toScalaMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = {
- keys.zip(values).toMap
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayData.scala
deleted file mode 100644
index b4ea300..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayData.scala
+++ /dev/null
@@ -1,139 +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 scala.reflect.ClassTag
-
-import org.apache.spark.sql.catalyst.expressions.SpecializedGetters
-
-abstract class ArrayData extends SpecializedGetters with Serializable {
- def numElements(): Int
-
- def copy(): ArrayData
-
- def array: Array[Any]
-
- def toBooleanArray(): Array[Boolean] = {
- val size = numElements()
- val values = new Array[Boolean](size)
- var i = 0
- while (i < size) {
- values(i) = getBoolean(i)
- i += 1
- }
- values
- }
-
- def toByteArray(): Array[Byte] = {
- val size = numElements()
- val values = new Array[Byte](size)
- var i = 0
- while (i < size) {
- values(i) = getByte(i)
- i += 1
- }
- values
- }
-
- def toShortArray(): Array[Short] = {
- val size = numElements()
- val values = new Array[Short](size)
- var i = 0
- while (i < size) {
- values(i) = getShort(i)
- i += 1
- }
- values
- }
-
- def toIntArray(): Array[Int] = {
- val size = numElements()
- val values = new Array[Int](size)
- var i = 0
- while (i < size) {
- values(i) = getInt(i)
- i += 1
- }
- values
- }
-
- def toLongArray(): Array[Long] = {
- val size = numElements()
- val values = new Array[Long](size)
- var i = 0
- while (i < size) {
- values(i) = getLong(i)
- i += 1
- }
- values
- }
-
- def toFloatArray(): Array[Float] = {
- val size = numElements()
- val values = new Array[Float](size)
- var i = 0
- while (i < size) {
- values(i) = getFloat(i)
- i += 1
- }
- values
- }
-
- def toDoubleArray(): Array[Double] = {
- val size = numElements()
- val values = new Array[Double](size)
- var i = 0
- while (i < size) {
- values(i) = getDouble(i)
- i += 1
- }
- values
- }
-
- def toObjectArray(elementType: DataType): Array[AnyRef] =
- toArray[AnyRef](elementType: DataType)
-
- def toArray[T: ClassTag](elementType: DataType): Array[T] = {
- val size = numElements()
- val values = new Array[T](size)
- var i = 0
- while (i < size) {
- if (isNullAt(i)) {
- values(i) = null.asInstanceOf[T]
- } else {
- values(i) = get(i, elementType).asInstanceOf[T]
- }
- i += 1
- }
- values
- }
-
- // todo: specialize this.
- def foreach(elementType: DataType, f: (Int, Any) => Unit): Unit = {
- val size = numElements()
- var i = 0
- while (i < size) {
- if (isNullAt(i)) {
- f(i, null)
- } else {
- f(i, get(i, elementType))
- }
- i += 1
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala
deleted file mode 100644
index 6e081ea..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala
+++ /dev/null
@@ -1,115 +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 scala.language.implicitConversions
-import scala.util.matching.Regex
-import scala.util.parsing.combinator.syntactical.StandardTokenParsers
-
-import org.apache.spark.sql.catalyst.SqlLexical
-
-/**
- * This is a data type parser that can be used to parse string representations of data types
- * provided in SQL queries. This parser is mixed in with DDLParser and SqlParser.
- */
-private[sql] trait DataTypeParser extends StandardTokenParsers {
-
- // This is used to create a parser from a regex. We are using regexes for data type strings
- // since these strings can be also used as column names or field names.
- import lexical.Identifier
- implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch(
- s"identifier matching regex ${regex}",
- { case Identifier(str) if regex.unapplySeq(str).isDefined => str }
- )
-
- protected lazy val primitiveType: Parser[DataType] =
- "(?i)string".r ^^^ StringType |
- "(?i)float".r ^^^ FloatType |
- "(?i)(?:int|integer)".r ^^^ IntegerType |
- "(?i)tinyint".r ^^^ ByteType |
- "(?i)smallint".r ^^^ ShortType |
- "(?i)double".r ^^^ DoubleType |
- "(?i)(?:bigint|long)".r ^^^ LongType |
- "(?i)binary".r ^^^ BinaryType |
- "(?i)boolean".r ^^^ BooleanType |
- fixedDecimalType |
- "(?i)decimal".r ^^^ DecimalType.USER_DEFAULT |
- "(?i)date".r ^^^ DateType |
- "(?i)timestamp".r ^^^ TimestampType |
- varchar
-
- protected lazy val fixedDecimalType: Parser[DataType] =
- ("(?i)decimal".r ~> "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ {
- case precision ~ scale =>
- DecimalType(precision.toInt, scale.toInt)
- }
-
- protected lazy val varchar: Parser[DataType] =
- "(?i)varchar".r ~> "(" ~> (numericLit <~ ")") ^^^ StringType
-
- protected lazy val arrayType: Parser[DataType] =
- "(?i)array".r ~> "<" ~> dataType <~ ">" ^^ {
- case tpe => ArrayType(tpe)
- }
-
- protected lazy val mapType: Parser[DataType] =
- "(?i)map".r ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ {
- case t1 ~ _ ~ t2 => MapType(t1, t2)
- }
-
- protected lazy val structField: Parser[StructField] =
- ident ~ ":" ~ dataType ^^ {
- case name ~ _ ~ tpe => StructField(name, tpe, nullable = true)
- }
-
- protected lazy val structType: Parser[DataType] =
- ("(?i)struct".r ~> "<" ~> repsep(structField, ",") <~ ">" ^^ {
- case fields => new StructType(fields.toArray)
- }) |
- ("(?i)struct".r ~ "<>" ^^^ StructType(Nil))
-
- protected lazy val dataType: Parser[DataType] =
- arrayType |
- mapType |
- structType |
- primitiveType
-
- def toDataType(dataTypeString: String): DataType = synchronized {
- phrase(dataType)(new lexical.Scanner(dataTypeString)) match {
- case Success(result, _) => result
- case failure: NoSuccess => throw new DataTypeException(failMessage(dataTypeString))
- }
- }
-
- private def failMessage(dataTypeString: String): String = {
- s"Unsupported dataType: $dataTypeString. If you have a struct and a field name of it has " +
- "any special characters, please use backticks (`) to quote that field name, e.g. `x+y`. " +
- "Please note that backtick itself is not supported in a field name."
- }
-}
-
-private[sql] object DataTypeParser {
- lazy val dataTypeParser = new DataTypeParser {
- override val lexical = new SqlLexical
- }
-
- def parse(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString)
-}
-
-/** The exception thrown from the [[DataTypeParser]]. */
-private[sql] class DataTypeException(message: String) extends Exception(message)
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/GenericArrayData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/GenericArrayData.scala
deleted file mode 100644
index 9448d88..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/GenericArrayData.scala
+++ /dev/null
@@ -1,135 +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.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
-
-class GenericArrayData(val array: Array[Any]) extends ArrayData {
-
- def this(seq: scala.collection.GenIterable[Any]) = this(seq.toArray)
-
- // TODO: This is boxing. We should specialize.
- def this(primitiveArray: Array[Int]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Long]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Float]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Double]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Short]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq)
- def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq)
-
- override def copy(): ArrayData = new GenericArrayData(array.clone())
-
- override def numElements(): Int = array.length
-
- private def getAs[T](ordinal: Int) = array(ordinal).asInstanceOf[T]
- override def isNullAt(ordinal: Int): Boolean = getAs[AnyRef](ordinal) eq null
- override def get(ordinal: Int, elementType: DataType): AnyRef = getAs(ordinal)
- override def getBoolean(ordinal: Int): Boolean = getAs(ordinal)
- override def getByte(ordinal: Int): Byte = getAs(ordinal)
- override def getShort(ordinal: Int): Short = getAs(ordinal)
- override def getInt(ordinal: Int): Int = getAs(ordinal)
- override def getLong(ordinal: Int): Long = getAs(ordinal)
- override def getFloat(ordinal: Int): Float = getAs(ordinal)
- override def getDouble(ordinal: Int): Double = getAs(ordinal)
- override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = getAs(ordinal)
- override def getUTF8String(ordinal: Int): UTF8String = getAs(ordinal)
- override def getBinary(ordinal: Int): Array[Byte] = getAs(ordinal)
- override def getInterval(ordinal: Int): CalendarInterval = getAs(ordinal)
- override def getStruct(ordinal: Int, numFields: Int): InternalRow = getAs(ordinal)
- override def getArray(ordinal: Int): ArrayData = getAs(ordinal)
- override def getMap(ordinal: Int): MapData = getAs(ordinal)
-
- override def toString(): String = array.mkString("[", ",", "]")
-
- override def equals(o: Any): Boolean = {
- if (!o.isInstanceOf[GenericArrayData]) {
- return false
- }
-
- val other = o.asInstanceOf[GenericArrayData]
- if (other eq null) {
- return false
- }
-
- val len = numElements()
- if (len != other.numElements()) {
- return false
- }
-
- var i = 0
- while (i < len) {
- if (isNullAt(i) != other.isNullAt(i)) {
- return false
- }
- if (!isNullAt(i)) {
- val o1 = array(i)
- val o2 = other.array(i)
- o1 match {
- case b1: Array[Byte] =>
- if (!o2.isInstanceOf[Array[Byte]] ||
- !java.util.Arrays.equals(b1, o2.asInstanceOf[Array[Byte]])) {
- return false
- }
- case f1: Float if java.lang.Float.isNaN(f1) =>
- if (!o2.isInstanceOf[Float] || ! java.lang.Float.isNaN(o2.asInstanceOf[Float])) {
- return false
- }
- case d1: Double if java.lang.Double.isNaN(d1) =>
- if (!o2.isInstanceOf[Double] || ! java.lang.Double.isNaN(o2.asInstanceOf[Double])) {
- return false
- }
- case _ => if (o1 != o2) {
- return false
- }
- }
- }
- i += 1
- }
- true
- }
-
- override def hashCode: Int = {
- var result: Int = 37
- var i = 0
- val len = numElements()
- while (i < len) {
- val update: Int =
- if (isNullAt(i)) {
- 0
- } else {
- array(i) match {
- case b: Boolean => if (b) 0 else 1
- case b: Byte => b.toInt
- case s: Short => s.toInt
- case i: Int => i
- case l: Long => (l ^ (l >>> 32)).toInt
- case f: Float => java.lang.Float.floatToIntBits(f)
- case d: Double =>
- val b = java.lang.Double.doubleToLongBits(d)
- (b ^ (b >>> 32)).toInt
- case a: Array[Byte] => java.util.Arrays.hashCode(a)
- case other => other.hashCode()
- }
- }
- result = 37 * result + update
- i += 1
- }
- result
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapData.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapData.scala
deleted file mode 100644
index f50969f..0000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapData.scala
+++ /dev/null
@@ -1,40 +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
-
-abstract class MapData extends Serializable {
-
- def numElements(): Int
-
- def keyArray(): ArrayData
-
- def valueArray(): ArrayData
-
- def copy(): MapData
-
- def foreach(keyType: DataType, valueType: DataType, f: (Any, Any) => Unit): Unit = {
- val length = numElements()
- val keys = keyArray()
- val values = valueArray()
- var i = 0
- while (i < length) {
- f(keys.get(i, keyType), values.get(i, valueType))
- i += 1
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
index d6b4367..11fce4b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -18,13 +18,13 @@
package org.apache.spark.sql.types
import scala.collection.mutable.ArrayBuffer
-import scala.math.max
import org.json4s.JsonDSL._
import org.apache.spark.SparkException
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering}
+import org.apache.spark.sql.catalyst.util.DataTypeParser
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ProductEncoderSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ProductEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ProductEncoderSuite.scala
index 7735acb..008d0be 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ProductEncoderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ProductEncoderSuite.scala
@@ -17,15 +17,13 @@
package org.apache.spark.sql.catalyst.encoders
-import java.util
-
-import org.apache.spark.sql.types.{StructField, ArrayType, ArrayData}
-
import scala.collection.mutable.ArrayBuffer
import scala.reflect.runtime.universe._
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst._
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.types.{StructField, ArrayType}
case class RepeatedStruct(s: Seq[PrimitiveData])
@@ -166,43 +164,43 @@ class ProductEncoderSuite extends SparkFunSuite {
null: Array[Byte]))
encodeDecodeTestCustom(("Array[Byte]",
Array[Byte](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Int] null",
null: Array[Int]))
encodeDecodeTestCustom(("Array[Int]",
Array[Int](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Long] null",
null: Array[Long]))
encodeDecodeTestCustom(("Array[Long]",
Array[Long](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Double] null",
null: Array[Double]))
encodeDecodeTestCustom(("Array[Double]",
Array[Double](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Float] null",
null: Array[Float]))
encodeDecodeTestCustom(("Array[Float]",
Array[Float](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Boolean] null",
null: Array[Boolean]))
encodeDecodeTestCustom(("Array[Boolean]",
Array[Boolean](true, false)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTest(("Array[Short] null",
null: Array[Short]))
encodeDecodeTestCustom(("Array[Short]",
Array[Short](1, 2, 3)))
- { (l, r) => util.Arrays.equals(l._2, r._2) }
+ { (l, r) => java.util.Arrays.equals(l._2, r._2) }
encodeDecodeTestCustom(("java.sql.Timestamp",
new java.sql.Timestamp(1)))
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
index c6aad34..68545f3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
@@ -23,7 +23,7 @@ import org.scalatest.Matchers
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
index 098944a..5adcac3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DataTypeParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DataTypeParserSuite.scala
new file mode 100644
index 0000000..1e3409a
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DataTypeParserSuite.scala
@@ -0,0 +1,119 @@
+/*
+* 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.catalyst.util
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.types._
+
+class DataTypeParserSuite extends SparkFunSuite {
+
+ def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = {
+ test(s"parse ${dataTypeString.replace("\n", "")}") {
+ assert(DataTypeParser.parse(dataTypeString) === expectedDataType)
+ }
+ }
+
+ def unsupported(dataTypeString: String): Unit = {
+ test(s"$dataTypeString is not supported") {
+ intercept[DataTypeException](DataTypeParser.parse(dataTypeString))
+ }
+ }
+
+ checkDataType("int", IntegerType)
+ checkDataType("integer", IntegerType)
+ checkDataType("BooLean", BooleanType)
+ checkDataType("tinYint", ByteType)
+ checkDataType("smallINT", ShortType)
+ checkDataType("INT", IntegerType)
+ checkDataType("INTEGER", IntegerType)
+ checkDataType("bigint", LongType)
+ checkDataType("float", FloatType)
+ checkDataType("dOUBle", DoubleType)
+ checkDataType("decimal(10, 5)", DecimalType(10, 5))
+ checkDataType("decimal", DecimalType.USER_DEFAULT)
+ checkDataType("DATE", DateType)
+ checkDataType("timestamp", TimestampType)
+ checkDataType("string", StringType)
+ checkDataType("varchAr(20)", StringType)
+ checkDataType("BINARY", BinaryType)
+
+ checkDataType("array<doublE>", ArrayType(DoubleType, true))
+ checkDataType("Array<map<int, tinYint>>", ArrayType(MapType(IntegerType, ByteType, true), true))
+ checkDataType(
+ "array<struct<tinYint:tinyint>>",
+ ArrayType(StructType(StructField("tinYint", ByteType, true) :: Nil), true)
+ )
+ checkDataType("MAP<int, STRING>", MapType(IntegerType, StringType, true))
+ checkDataType("MAp<int, ARRAY<double>>", MapType(IntegerType, ArrayType(DoubleType), true))
+ checkDataType(
+ "MAP<int, struct<varchar:string>>",
+ MapType(IntegerType, StructType(StructField("varchar", StringType, true) :: Nil), true)
+ )
+
+ checkDataType(
+ "struct<intType: int, ts:timestamp>",
+ StructType(
+ StructField("intType", IntegerType, true) ::
+ StructField("ts", TimestampType, true) :: Nil)
+ )
+ // It is fine to use the data type string as the column name.
+ checkDataType(
+ "Struct<int: int, timestamp:timestamp>",
+ StructType(
+ StructField("int", IntegerType, true) ::
+ StructField("timestamp", TimestampType, true) :: Nil)
+ )
+ checkDataType(
+ """
+ |struct<
+ | struct:struct<deciMal:DECimal, anotherDecimal:decimAL(5,2)>,
+ | MAP:Map<timestamp, varchar(10)>,
+ | arrAy:Array<double>>
+ """.stripMargin,
+ StructType(
+ StructField("struct",
+ StructType(
+ StructField("deciMal", DecimalType.USER_DEFAULT, true) ::
+ StructField("anotherDecimal", DecimalType(5, 2), true) :: Nil), true) ::
+ StructField("MAP", MapType(TimestampType, StringType), true) ::
+ StructField("arrAy", ArrayType(DoubleType, true), true) :: Nil)
+ )
+ // A column name can be a reserved word in our DDL parser and SqlParser.
+ checkDataType(
+ "Struct<TABLE: string, CASE:boolean>",
+ StructType(
+ StructField("TABLE", StringType, true) ::
+ StructField("CASE", BooleanType, true) :: Nil)
+ )
+ // Use backticks to quote column names having special characters.
+ checkDataType(
+ "struct<`x+y`:int, `!@#$%^&*()`:string, `1_2.345<>:\"`:varchar(20)>",
+ StructType(
+ StructField("x+y", IntegerType, true) ::
+ StructField("!@#$%^&*()", StringType, true) ::
+ StructField("1_2.345<>:\"", StringType, true) :: Nil)
+ )
+ // Empty struct.
+ checkDataType("strUCt<>", StructType(Nil))
+
+ unsupported("it is not a data type")
+ unsupported("struct<x+y: int, 1.1:timestamp>")
+ unsupported("struct<x: int")
+ unsupported("struct<x int, y string>")
+ unsupported("struct<`x``y` int>")
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/cdea0174/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala
deleted file mode 100644
index 1ba2907..0000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala
+++ /dev/null
@@ -1,118 +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.apache.spark.SparkFunSuite
-
-class DataTypeParserSuite extends SparkFunSuite {
-
- def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = {
- test(s"parse ${dataTypeString.replace("\n", "")}") {
- assert(DataTypeParser.parse(dataTypeString) === expectedDataType)
- }
- }
-
- def unsupported(dataTypeString: String): Unit = {
- test(s"$dataTypeString is not supported") {
- intercept[DataTypeException](DataTypeParser.parse(dataTypeString))
- }
- }
-
- checkDataType("int", IntegerType)
- checkDataType("integer", IntegerType)
- checkDataType("BooLean", BooleanType)
- checkDataType("tinYint", ByteType)
- checkDataType("smallINT", ShortType)
- checkDataType("INT", IntegerType)
- checkDataType("INTEGER", IntegerType)
- checkDataType("bigint", LongType)
- checkDataType("float", FloatType)
- checkDataType("dOUBle", DoubleType)
- checkDataType("decimal(10, 5)", DecimalType(10, 5))
- checkDataType("decimal", DecimalType.USER_DEFAULT)
- checkDataType("DATE", DateType)
- checkDataType("timestamp", TimestampType)
- checkDataType("string", StringType)
- checkDataType("varchAr(20)", StringType)
- checkDataType("BINARY", BinaryType)
-
- checkDataType("array<doublE>", ArrayType(DoubleType, true))
- checkDataType("Array<map<int, tinYint>>", ArrayType(MapType(IntegerType, ByteType, true), true))
- checkDataType(
- "array<struct<tinYint:tinyint>>",
- ArrayType(StructType(StructField("tinYint", ByteType, true) :: Nil), true)
- )
- checkDataType("MAP<int, STRING>", MapType(IntegerType, StringType, true))
- checkDataType("MAp<int, ARRAY<double>>", MapType(IntegerType, ArrayType(DoubleType), true))
- checkDataType(
- "MAP<int, struct<varchar:string>>",
- MapType(IntegerType, StructType(StructField("varchar", StringType, true) :: Nil), true)
- )
-
- checkDataType(
- "struct<intType: int, ts:timestamp>",
- StructType(
- StructField("intType", IntegerType, true) ::
- StructField("ts", TimestampType, true) :: Nil)
- )
- // It is fine to use the data type string as the column name.
- checkDataType(
- "Struct<int: int, timestamp:timestamp>",
- StructType(
- StructField("int", IntegerType, true) ::
- StructField("timestamp", TimestampType, true) :: Nil)
- )
- checkDataType(
- """
- |struct<
- | struct:struct<deciMal:DECimal, anotherDecimal:decimAL(5,2)>,
- | MAP:Map<timestamp, varchar(10)>,
- | arrAy:Array<double>>
- """.stripMargin,
- StructType(
- StructField("struct",
- StructType(
- StructField("deciMal", DecimalType.USER_DEFAULT, true) ::
- StructField("anotherDecimal", DecimalType(5, 2), true) :: Nil), true) ::
- StructField("MAP", MapType(TimestampType, StringType), true) ::
- StructField("arrAy", ArrayType(DoubleType, true), true) :: Nil)
- )
- // A column name can be a reserved word in our DDL parser and SqlParser.
- checkDataType(
- "Struct<TABLE: string, CASE:boolean>",
- StructType(
- StructField("TABLE", StringType, true) ::
- StructField("CASE", BooleanType, true) :: Nil)
- )
- // Use backticks to quote column names having special characters.
- checkDataType(
- "struct<`x+y`:int, `!@#$%^&*()`:string, `1_2.345<>:\"`:varchar(20)>",
- StructType(
- StructField("x+y", IntegerType, true) ::
- StructField("!@#$%^&*()", StringType, true) ::
- StructField("1_2.345<>:\"", StringType, true) :: Nil)
- )
- // Empty struct.
- checkDataType("strUCt<>", StructType(Nil))
-
- unsupported("it is not a data type")
- unsupported("struct<x+y: int, 1.1:timestamp>")
- unsupported("struct<x: int")
- unsupported("struct<x int, y string>")
- unsupported("struct<`x``y` int>")
-}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org