Skip to content

Commit b8b7db4

Browse files
committed
1. Move sql package object and package-info to sql-core.
2. Minor updates on APIs. 3. Update scala doc.
1 parent 68525a2 commit b8b7db4

File tree

14 files changed

+438
-161
lines changed

14 files changed

+438
-161
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,11 +17,12 @@
1717

1818
package org.apache.spark.sql.catalyst.expressions
1919

20+
import com.typesafe.scalalogging.slf4j.Logging
21+
2022
import org.apache.spark.sql.catalyst.trees
2123
import org.apache.spark.sql.catalyst.errors.attachTree
2224
import org.apache.spark.sql.catalyst.plans.QueryPlan
2325
import org.apache.spark.sql.catalyst.rules.Rule
24-
import org.apache.spark.sql.Logging
2526

2627
/**
2728
* A bound reference points to a specific slot in the input tuple, allowing the actual value

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ object Row {
3434
def unapplySeq(row: Row): Some[Seq[Any]] = Some(row)
3535

3636
/**
37-
* Construct a [[Row]] with the given values.
37+
* This method can be used to construct a [[Row]] with the given values.
3838
*/
3939
def apply(values: Any*): Row = new GenericRow(values.toArray)
4040
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,8 @@
1717

1818
package org.apache.spark.sql.catalyst.planning
1919

20-
import org.apache.spark.sql.Logging
20+
import com.typesafe.scalalogging.slf4j.Logging
21+
2122
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2223
import org.apache.spark.sql.catalyst.trees.TreeNode
2324

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.planning
1919

2020
import scala.annotation.tailrec
2121

22-
import org.apache.spark.sql.Logging
22+
import com.typesafe.scalalogging.slf4j.Logging
2323

2424
import org.apache.spark.sql.catalyst.expressions._
2525
import org.apache.spark.sql.catalyst.plans._

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy
128128
def schema: StructType = StructType.fromAttributes(output)
129129

130130
/** Returns the output schema in the tree format. */
131-
def schemaString: String = schema.schemaString
131+
def schemaString: String = schema.structString
132132

133133
/** Prints out the schema in the tree format */
134134
def printSchema(): Unit = println(schemaString)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,8 @@
1717

1818
package org.apache.spark.sql.catalyst.rules
1919

20-
import org.apache.spark.sql.Logging
20+
import com.typesafe.scalalogging.slf4j.Logging
21+
2122
import org.apache.spark.sql.catalyst.trees.TreeNode
2223

2324
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -15,9 +15,9 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql
19-
package catalyst
20-
package rules
18+
package org.apache.spark.sql.catalyst.rules
19+
20+
import com.typesafe.scalalogging.slf4j.Logging
2121

2222
import org.apache.spark.sql.catalyst.trees.TreeNode
2323
import org.apache.spark.sql.catalyst.util.sideBySide

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,6 @@
1717

1818
package org.apache.spark.sql.catalyst
1919

20-
import org.apache.spark.sql.Logger
21-
2220
/**
2321
* A library for easily manipulating trees of operators. Operators that extend TreeNode are
2422
* granted the following interface:
@@ -35,5 +33,6 @@ import org.apache.spark.sql.Logger
3533
*/
3634
package object trees {
3735
// Since we want tree nodes to be lightweight, we create one logger for all treenode instances.
38-
protected val logger = Logger("catalyst.trees")
36+
protected val logger =
37+
com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees"))
3938
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala

Lines changed: 36 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.util.Utils
3030
/**
3131
*
3232
*/
33-
object DataType extends RegexParsers {
33+
protected[sql] object DataType extends RegexParsers {
3434
protected lazy val primitiveType: Parser[DataType] =
3535
"StringType" ^^^ StringType |
3636
"FloatType" ^^^ FloatType |
@@ -84,6 +84,21 @@ object DataType extends RegexParsers {
8484
case Success(result, _) => result
8585
case failure: NoSuccess => sys.error(s"Unsupported dataType: $asString, $failure")
8686
}
87+
88+
protected[types] def buildFormattedString(
89+
dataType: DataType,
90+
prefix: String,
91+
builder: StringBuilder): Unit = {
92+
dataType match {
93+
case array: ArrayType =>
94+
array.buildFormattedString(prefix, builder)
95+
case struct: StructType =>
96+
struct.buildFormattedString(prefix, builder)
97+
case map: MapType =>
98+
map.buildFormattedString(prefix, builder)
99+
case _ =>
100+
}
101+
}
87102
}
88103

89104
abstract class DataType {
@@ -244,22 +259,15 @@ case object FloatType extends FractionalType {
244259
}
245260

246261
object ArrayType {
262+
/** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */
247263
def apply(elementType: DataType): ArrayType = ArrayType(elementType, false)
248264
}
249265

250266
case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType {
251267
private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
252268
builder.append(
253269
s"${prefix}-- element: ${elementType.simpleString} (containsNull = ${containsNull})\n")
254-
elementType match {
255-
case array: ArrayType =>
256-
array.buildFormattedString(s"$prefix |", builder)
257-
case struct: StructType =>
258-
struct.buildFormattedString(s"$prefix |", builder)
259-
case map: MapType =>
260-
map.buildFormattedString(s"$prefix |", builder)
261-
case _ =>
262-
}
270+
DataType.buildFormattedString(elementType, s"$prefix |", builder)
263271
}
264272

265273
def simpleString: String = "array"
@@ -269,48 +277,41 @@ case class StructField(name: String, dataType: DataType, nullable: Boolean) {
269277

270278
private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
271279
builder.append(s"${prefix}-- ${name}: ${dataType.simpleString} (nullable = ${nullable})\n")
272-
dataType match {
273-
case array: ArrayType =>
274-
array.buildFormattedString(s"$prefix |", builder)
275-
case struct: StructType =>
276-
struct.buildFormattedString(s"$prefix |", builder)
277-
case map: MapType =>
278-
map.buildFormattedString(s"$prefix |", builder)
279-
case _ =>
280-
}
280+
DataType.buildFormattedString(dataType, s"$prefix |", builder)
281281
}
282282
}
283283

284284
object StructType {
285-
def fromAttributes(attributes: Seq[Attribute]): StructType =
285+
protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
286286
StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable)))
287287

288288
private def validateFields(fields: Seq[StructField]): Boolean =
289289
fields.map(field => field.name).distinct.size == fields.size
290-
291-
def apply[A <: String: ClassTag, B <: DataType: ClassTag](fields: (A, B)*): StructType =
292-
StructType(fields.map(field => StructField(field._1, field._2, true)))
293-
294-
def apply[A <: String: ClassTag, B <: DataType: ClassTag, C <: Boolean: ClassTag](
295-
fields: (A, B, C)*): StructType =
296-
StructType(fields.map(field => StructField(field._1, field._2, field._3)))
297290
}
298291

299292
case class StructType(fields: Seq[StructField]) extends DataType {
300293
require(StructType.validateFields(fields), "Found fields with the same name.")
301294

295+
/**
296+
* Extracts a [[StructField]] of the given name. If the [[StructType]] object does not
297+
* have a name matching the given name, `null` will be returned.
298+
*/
302299
def apply(name: String): StructField = {
303300
fields.find(f => f.name == name).orNull
304301
}
305302

306-
def apply(names: String*): StructType = {
307-
val nameSet = names.toSet
308-
StructType(fields.filter(f => nameSet.contains(f.name)))
303+
/**
304+
* Returns a [[StructType]] containing [[StructField]]s of the given names.
305+
* Those names which do not have matching fields will be ignored.
306+
*/
307+
def apply(names: Set[String]): StructType = {
308+
StructType(fields.filter(f => names.contains(f.name)))
309309
}
310310

311-
def toAttributes = fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)())
311+
protected[sql] def toAttributes =
312+
fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)())
312313

313-
def schemaString: String = {
314+
def structString: String = {
314315
val builder = new StringBuilder
315316
builder.append("root\n")
316317
val prefix = " |"
@@ -319,7 +320,7 @@ case class StructType(fields: Seq[StructField]) extends DataType {
319320
builder.toString()
320321
}
321322

322-
def printSchema(): Unit = println(schemaString)
323+
def printStruct(): Unit = println(structString)
323324

324325
private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
325326
fields.foreach(field => field.buildFormattedString(prefix, builder))
@@ -331,26 +332,8 @@ case class StructType(fields: Seq[StructField]) extends DataType {
331332
case class MapType(keyType: DataType, valueType: DataType) extends DataType {
332333
private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = {
333334
builder.append(s"${prefix}-- key: ${keyType.simpleString}\n")
334-
keyType match {
335-
case array: ArrayType =>
336-
array.buildFormattedString(s"$prefix |", builder)
337-
case struct: StructType =>
338-
struct.buildFormattedString(s"$prefix |", builder)
339-
case map: MapType =>
340-
map.buildFormattedString(s"$prefix |", builder)
341-
case _ =>
342-
}
343-
344-
builder.append(s"${prefix}-- value: ${valueType.simpleString}\n")
345-
valueType match {
346-
case array: ArrayType =>
347-
array.buildFormattedString(s"$prefix |", builder)
348-
case struct: StructType =>
349-
struct.buildFormattedString(s"$prefix |", builder)
350-
case map: MapType =>
351-
map.buildFormattedString(s"$prefix |", builder)
352-
case _ =>
353-
}
335+
DataType.buildFormattedString(keyType, s"$prefix |", builder)
336+
DataType.buildFormattedString(valueType, s"$prefix |", builder)
354337
}
355338

356339
def simpleString: String = "map"

sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala

Lines changed: 0 additions & 80 deletions
This file was deleted.

0 commit comments

Comments
 (0)