Skip to content

Commit

Permalink
[SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.…
Browse files Browse the repository at this point in the history
…util package

Author: Reynold Xin <[email protected]>

Closes apache#9239 from rxin/types-private.
  • Loading branch information
rxin committed Oct 23, 2015
1 parent b1c1597 commit cdea017
Show file tree
Hide file tree
Showing 50 changed files with 76 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

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;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@

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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._

////////////////////////////////////////////////////////////////////////////////////////////////////
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.sql
package org.apache.spark.sql.catalyst.util

/**
* Shim to allow us to implement [[scala.Iterator]] in Java. Scala 2.11+ has an AbstractIterator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
package org.apache.spark.sql.catalyst.util

class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData {
require(keyArray.numElements() == valueArray.numElements())
Expand All @@ -42,7 +42,7 @@ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) exte
ArrayBasedMapData.toScalaMap(this).hashCode()
}

override def toString(): String = {
override def toString: String = {
s"keys: $keyArray, values: $valueArray"
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,12 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,14 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,10 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,9 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
package org.apache.spark.sql.catalyst.util

import org.apache.spark.sql.types.DataType

abstract class MapData extends Serializable {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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


/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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])

Expand Down Expand Up @@ -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)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,10 @@
* limitations under the License.
*/

package org.apache.spark.sql.types
package org.apache.spark.sql.catalyst.util

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.types._

class DataTypeParserSuite extends SparkFunSuite {

Expand Down
Loading

0 comments on commit cdea017

Please sign in to comment.