Skip to content

Commit bd40a33

Browse files
committed
Address comments.
1 parent 991f860 commit bd40a33

File tree

28 files changed

+154
-78
lines changed

28 files changed

+154
-78
lines changed

python/pyspark/sql.py

Lines changed: 52 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -41,18 +41,18 @@ class StringType(object):
4141
"""
4242
__metaclass__ = PrimitiveTypeSingleton
4343

44-
def _get_scala_type_string(self):
44+
def __repr__(self):
4545
return "StringType"
4646

4747
class BinaryType(object):
4848
"""Spark SQL BinaryType
4949
50-
The data type representing bytes values and bytearray values.
50+
The data type representing bytearray values.
5151
5252
"""
5353
__metaclass__ = PrimitiveTypeSingleton
5454

55-
def _get_scala_type_string(self):
55+
def __repr__(self):
5656
return "BinaryType"
5757

5858
class BooleanType(object):
@@ -63,14 +63,18 @@ class BooleanType(object):
6363
"""
6464
__metaclass__ = PrimitiveTypeSingleton
6565

66-
def _get_scala_type_string(self):
66+
def __repr__(self):
6767
return "BooleanType"
6868

6969
class TimestampType(object):
70-
"""Spark SQL TimestampType"""
70+
"""Spark SQL TimestampType
71+
72+
The data type representing datetime.datetime values.
73+
74+
"""
7175
__metaclass__ = PrimitiveTypeSingleton
7276

73-
def _get_scala_type_string(self):
77+
def __repr__(self):
7478
return "TimestampType"
7579

7680
class DecimalType(object):
@@ -81,40 +85,48 @@ class DecimalType(object):
8185
"""
8286
__metaclass__ = PrimitiveTypeSingleton
8387

84-
def _get_scala_type_string(self):
88+
def __repr__(self):
8589
return "DecimalType"
8690

8791
class DoubleType(object):
8892
"""Spark SQL DoubleType
8993
90-
The data type representing float values. Because a float value
94+
The data type representing float values.
9195
9296
"""
9397
__metaclass__ = PrimitiveTypeSingleton
9498

95-
def _get_scala_type_string(self):
99+
def __repr__(self):
96100
return "DoubleType"
97101

98102
class FloatType(object):
99103
"""Spark SQL FloatType
100104
101-
For PySpark, please use L{DoubleType} instead of using L{FloatType}.
105+
For now, please use L{DoubleType} instead of using L{FloatType}.
106+
Because query evaluation is done in Scala, java.lang.Double will be be used
107+
for Python float numbers. Because the underlying JVM type of FloatType is
108+
java.lang.Float (in Java) and Float (in scala), there will be a java.lang.ClassCastException
109+
if FloatType (Python) used.
102110
103111
"""
104112
__metaclass__ = PrimitiveTypeSingleton
105113

106-
def _get_scala_type_string(self):
114+
def __repr__(self):
107115
return "FloatType"
108116

109117
class ByteType(object):
110118
"""Spark SQL ByteType
111119
112-
For PySpark, please use L{IntegerType} instead of using L{ByteType}.
120+
For now, please use L{IntegerType} instead of using L{ByteType}.
121+
Because query evaluation is done in Scala, java.lang.Integer will be be used
122+
for Python int numbers. Because the underlying JVM type of ByteType is
123+
java.lang.Byte (in Java) and Byte (in scala), there will be a java.lang.ClassCastException
124+
if ByteType (Python) used.
113125
114126
"""
115127
__metaclass__ = PrimitiveTypeSingleton
116128

117-
def _get_scala_type_string(self):
129+
def __repr__(self):
118130
return "ByteType"
119131

120132
class IntegerType(object):
@@ -125,7 +137,7 @@ class IntegerType(object):
125137
"""
126138
__metaclass__ = PrimitiveTypeSingleton
127139

128-
def _get_scala_type_string(self):
140+
def __repr__(self):
129141
return "IntegerType"
130142

131143
class LongType(object):
@@ -137,18 +149,22 @@ class LongType(object):
137149
"""
138150
__metaclass__ = PrimitiveTypeSingleton
139151

140-
def _get_scala_type_string(self):
152+
def __repr__(self):
141153
return "LongType"
142154

143155
class ShortType(object):
144156
"""Spark SQL ShortType
145157
146-
For PySpark, please use L{IntegerType} instead of using L{ShortType}.
158+
For now, please use L{IntegerType} instead of using L{ShortType}.
159+
Because query evaluation is done in Scala, java.lang.Integer will be be used
160+
for Python int numbers. Because the underlying JVM type of ShortType is
161+
java.lang.Short (in Java) and Short (in scala), there will be a java.lang.ClassCastException
162+
if ShortType (Python) used.
147163
148164
"""
149165
__metaclass__ = PrimitiveTypeSingleton
150166

151-
def _get_scala_type_string(self):
167+
def __repr__(self):
152168
return "ShortType"
153169

154170
class ArrayType(object):
@@ -157,23 +173,23 @@ class ArrayType(object):
157173
The data type representing list values.
158174
159175
"""
160-
def __init__(self, elementType, containsNull):
176+
def __init__(self, elementType, containsNull=False):
161177
"""Creates an ArrayType
162178
163179
:param elementType: the data type of elements.
164180
:param containsNull: indicates whether the list contains null values.
165181
:return:
166182
167-
>>> ArrayType(StringType, True) == ArrayType(StringType, False)
168-
False
169-
>>> ArrayType(StringType, True) == ArrayType(StringType, True)
183+
>>> ArrayType(StringType) == ArrayType(StringType, False)
170184
True
185+
>>> ArrayType(StringType, True) == ArrayType(StringType)
186+
False
171187
"""
172188
self.elementType = elementType
173189
self.containsNull = containsNull
174190

175-
def _get_scala_type_string(self):
176-
return "ArrayType(" + self.elementType._get_scala_type_string() + "," + \
191+
def __repr__(self):
192+
return "ArrayType(" + self.elementType.__repr__() + "," + \
177193
str(self.containsNull).lower() + ")"
178194

179195
def __eq__(self, other):
@@ -207,9 +223,9 @@ def __init__(self, keyType, valueType, valueContainsNull=True):
207223
self.valueType = valueType
208224
self.valueContainsNull = valueContainsNull
209225

210-
def _get_scala_type_string(self):
211-
return "MapType(" + self.keyType._get_scala_type_string() + "," + \
212-
self.valueType._get_scala_type_string() + "," + \
226+
def __repr__(self):
227+
return "MapType(" + self.keyType.__repr__() + "," + \
228+
self.valueType.__repr__() + "," + \
213229
str(self.valueContainsNull).lower() + ")"
214230

215231
def __eq__(self, other):
@@ -243,9 +259,9 @@ def __init__(self, name, dataType, nullable):
243259
self.dataType = dataType
244260
self.nullable = nullable
245261

246-
def _get_scala_type_string(self):
262+
def __repr__(self):
247263
return "StructField(" + self.name + "," + \
248-
self.dataType._get_scala_type_string() + "," + \
264+
self.dataType.__repr__() + "," + \
249265
str(self.nullable).lower() + ")"
250266

251267
def __eq__(self, other):
@@ -280,9 +296,9 @@ def __init__(self, fields):
280296
"""
281297
self.fields = fields
282298

283-
def _get_scala_type_string(self):
299+
def __repr__(self):
284300
return "StructType(List(" + \
285-
",".join([field._get_scala_type_string() for field in self.fields]) + "))"
301+
",".join([field.__repr__() for field in self.fields]) + "))"
286302

287303
def __eq__(self, other):
288304
return (isinstance(other, self.__class__) and \
@@ -319,7 +335,7 @@ def _parse_datatype_string(datatype_string):
319335
:return:
320336
321337
>>> def check_datatype(datatype):
322-
... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype._get_scala_type_string())
338+
... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.__repr__())
323339
... python_datatype = _parse_datatype_string(scala_datatype.toString())
324340
... return datatype == python_datatype
325341
>>> check_datatype(StringType())
@@ -536,7 +552,7 @@ def applySchema(self, rdd, schema):
536552
True
537553
"""
538554
jrdd = self._pythonToJavaMap(rdd._jrdd)
539-
srdd = self._ssql_ctx.applySchema(jrdd.rdd(), schema._get_scala_type_string())
555+
srdd = self._ssql_ctx.applySchema(jrdd.rdd(), schema.__repr__())
540556
return SchemaRDD(srdd, self)
541557

542558
def registerRDDAsTable(self, rdd, tableName):
@@ -569,7 +585,7 @@ def parquetFile(self, path):
569585
jschema_rdd = self._ssql_ctx.parquetFile(path)
570586
return SchemaRDD(jschema_rdd, self)
571587

572-
def jsonFile(self, path, schema = None):
588+
def jsonFile(self, path, schema=None):
573589
"""Loads a text file storing one JSON object per line as a L{SchemaRDD}.
574590
575591
If the schema is provided, applies the given schema to this JSON dataset.
@@ -618,11 +634,11 @@ def jsonFile(self, path, schema = None):
618634
if schema is None:
619635
jschema_rdd = self._ssql_ctx.jsonFile(path)
620636
else:
621-
scala_datatype = self._ssql_ctx.parseDataType(schema._get_scala_type_string())
637+
scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__())
622638
jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype)
623639
return SchemaRDD(jschema_rdd, self)
624640

625-
def jsonRDD(self, rdd, schema = None):
641+
def jsonRDD(self, rdd, schema=None):
626642
"""Loads an RDD storing one JSON object per string as a L{SchemaRDD}.
627643
628644
If the schema is provided, applies the given schema to this JSON dataset.
@@ -672,7 +688,7 @@ def func(split, iterator):
672688
if schema is None:
673689
jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd())
674690
else:
675-
scala_datatype = self._ssql_ctx.parseDataType(schema._get_scala_type_string())
691+
scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__())
676692
jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype)
677693
return SchemaRDD(jschema_rdd, self)
678694

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
109109
object ResolveReferences extends Rule[LogicalPlan] {
110110
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
111111
case q: LogicalPlan if q.childrenResolved =>
112-
logger.trace(s"Attempting to resolve ${q.simpleString}")
112+
logTrace(s"Attempting to resolve ${q.simpleString}")
113113
q transformExpressions {
114114
case u @ UnresolvedAttribute(name) =>
115115
// Leave unchanged if resolution fails. Hopefully will be resolved next round.
116116
val result = q.resolve(name).getOrElse(u)
117-
logger.debug(s"Resolving $u to $result")
117+
logDebug(s"Resolving $u to $result")
118118
result
119119
}
120120
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ trait HiveTypeCoercion {
7575
// Leave the same if the dataTypes match.
7676
case Some(newType) if a.dataType == newType.dataType => a
7777
case Some(newType) =>
78-
logger.debug(s"Promoting $a to $newType in ${q.simpleString}}")
78+
logDebug(s"Promoting $a to $newType in ${q.simpleString}}")
7979
newType
8080
}
8181
}
@@ -154,7 +154,7 @@ trait HiveTypeCoercion {
154154
(Alias(Cast(l, StringType), l.name)(), r)
155155

156156
case (l, r) if l.dataType != r.dataType =>
157-
logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
157+
logDebug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}")
158158
findTightestCommonType(l.dataType, r.dataType).map { widestType =>
159159
val newLeft =
160160
if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)()
@@ -170,15 +170,15 @@ trait HiveTypeCoercion {
170170

171171
val newLeft =
172172
if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) {
173-
logger.debug(s"Widening numeric types in union $castedLeft ${left.output}")
173+
logDebug(s"Widening numeric types in union $castedLeft ${left.output}")
174174
Project(castedLeft, left)
175175
} else {
176176
left
177177
}
178178

179179
val newRight =
180180
if (castedRight.map(_.dataType) != right.output.map(_.dataType)) {
181-
logger.debug(s"Widening numeric types in union $castedRight ${right.output}")
181+
logDebug(s"Widening numeric types in union $castedRight ${right.output}")
182182
Project(castedRight, right)
183183
} else {
184184
right

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

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

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

20-
import com.typesafe.scalalogging.slf4j.Logging
21-
20+
import org.apache.spark.Logging
2221
import org.apache.spark.sql.catalyst.trees
2322
import org.apache.spark.sql.catalyst.errors.attachTree
2423
import org.apache.spark.sql.catalyst.plans.QueryPlan
@@ -80,7 +79,7 @@ object BindReferences extends Logging {
8079
// produce new attributes that can't be bound. Likely the right thing to do is remove
8180
// this rule and require all operators to explicitly bind to the input schema that
8281
// they specify.
83-
logger.debug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}")
82+
logDebug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}")
8483
a
8584
} else {
8685
BoundReference(ordinal, a)

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

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

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

20-
import com.typesafe.scalalogging.slf4j.Logging
21-
20+
import org.apache.spark.Logging
2221
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2322
import org.apache.spark.sql.catalyst.trees.TreeNode
2423

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

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst.planning
1919

2020
import scala.annotation.tailrec
2121

22-
import com.typesafe.scalalogging.slf4j.Logging
23-
22+
import org.apache.spark.Logging
2423
import org.apache.spark.sql.catalyst.expressions._
2524
import org.apache.spark.sql.catalyst.plans._
2625
import org.apache.spark.sql.catalyst.plans.logical._
@@ -114,7 +113,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
114113

115114
def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
116115
case join @ Join(left, right, joinType, condition) =>
117-
logger.debug(s"Considering join on: $condition")
116+
logDebug(s"Considering join on: $condition")
118117
// Find equi-join predicates that can be evaluated before the join, and thus can be used
119118
// as join keys.
120119
val (joinPredicates, otherPredicates) =
@@ -132,7 +131,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
132131
val rightKeys = joinKeys.map(_._2)
133132

134133
if (joinKeys.nonEmpty) {
135-
logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
134+
logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
136135
Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right))
137136
} else {
138137
None

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

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

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

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

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

0 commit comments

Comments
 (0)