@@ -32,31 +32,95 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal
3232import scala .collection .JavaConversions ._
3333
3434/**
35- * Hive ObjectInspector is a group of flexible APIs to inspect value in different data
36- * representation, (says: standard object <==> custom object). Developers can extend
37- * those API as needed, so technically, object inspector supports arbitrary data
38- * representation (convert to / from standard object)
35+ * 1. The Underlying data type in catalyst and in Hive
36+ * In catalyst:
37+ * Primitive =>
38+ * java.lang.String
39+ * int / scala.Int
40+ * boolean / scala.Boolean
41+ * float / scala.Float
42+ * double / scala.Double
43+ * long / scala.Long
44+ * short / scala.Short
45+ * byte / scala.Byte
46+ * org.apache.spark.sql.catalyst.types.decimal.Decimal
47+ * Array[Byte]
48+ * java.sql.Date
49+ * java.sql.Timestamp
50+ * Complicated Types =>
51+ * Map: scala.collection.immutable.Map
52+ * List: scala.collection.immutable.Seq
53+ * Struct:
54+ * org.apache.spark.sql.catalyst.expression.Row
55+ * org.apache.spark.sql.catalyst.expression.GenericRow
56+ * scala.collection.immutable.Seq
57+ * Union: NOT SUPPORTED YET
58+ * The Complicated types plays as a container, which can hold arbitrary data types.
3959 *
40- * HiveInspectors handles the most common used object inspectors, which are
41- * widely used in generic udf/udaf/udtf:
60+ * In Hive, the native data types are various, in UDF/UDAF/UDTF, and associated with
61+ * Object Inspectors, in Hive expression evaluation framework, the underlying data are
62+ * Primitive Type
63+ * Java Boxed Primitives:
64+ * org.apache.hadoop.hive.common.type.HiveVarchar
65+ * java.lang.String
66+ * java.lang.Integer
67+ * java.lang.Boolean
68+ * java.lang.Float
69+ * java.lang.Double
70+ * java.lang.Long
71+ * java.lang.Short
72+ * java.lang.Byte
73+ * org.apache.hadoop.hive.common.`type`.HiveDecimal
74+ * byte[]
75+ * java.sql.Date
76+ * java.sql.Timestamp
77+ * Writables:
78+ * org.apache.hadoop.hive.serde2.io.HiveVarcharWritable
79+ * org.apache.hadoop.io.Text
80+ * org.apache.hadoop.io.IntWritable
81+ * org.apache.hadoop.hive.serde2.io.DoubleWritable
82+ * org.apache.hadoop.io.BooleanWritable
83+ * org.apache.hadoop.io.LongWritable
84+ * org.apache.hadoop.io.FloatWritable
85+ * org.apache.hadoop.hive.serde2.io.ShortWritable
86+ * org.apache.hadoop.hive.serde2.io.ByteWritable
87+ * org.apache.hadoop.io.BytesWritable
88+ * org.apache.hadoop.hive.serde2.io.DateWritable
89+ * org.apache.hadoop.hive.serde2.io.TimestampWritable
90+ * org.apache.hadoop.hive.serde2.io.HiveDecimalWritable
91+ * Complicated Type
92+ * List: Object[] / java.util.List
93+ * Map: java.util.Map
94+ * Struct: Object[] / java.util.List / java POJO
95+ * Union: class StandardUnion { byte tag; Object object }
96+ *
97+ * NOTICE: HiveVarchar is not supported by catalyst, it will be simply considered as String type.
98+ *
99+ *
100+ * 2. Hive ObjectInspector is a group of flexible APIs to inspect value in different data
101+ * representation, and developers can extend those API as needed, so technically,
102+ * object inspector supports arbitrary data type in java.
103+ *
104+ * Fortunately, only few built-in Hive Object Inspectors are used in generic udf/udaf/udtf
105+ * evaluation.
42106 * 1) Primitive Types (PrimitiveObjectInspector & its sub classes)
43107 {{{
44108 public interface PrimitiveObjectInspector {
45109 // Java Primitives (java.lang.Integer, java.lang.String etc.)
46110 Object getPrimitiveWritableObject(Object o);
47111 // Writables (hadoop.io.IntWritable, hadoop.io.Text etc.)
48112 Object getPrimitiveJavaObject(Object o);
49- // Writable ObjectInspector always return true, we need to check that
113+ // ObjectInspector only inspect the `writable` always return true, we need to check it
50114 // before invoking the methods above.
51115 boolean preferWritable();
52116 ...
53117 }
54118 }}}
55119
56120 * 2) Complicated Types:
57- * ListObjectInspector: underlying data structure are java array or [[java.util.List ]]
58- * MapObjectInspector: underlying data structure are [[java.util.Map ]]
59- * Struct.StructObjectInspector: underlying data structure are java array, [[java.util.List ]] and
121+ * ListObjectInspector: inspects java array or [[java.util.List ]]
122+ * MapObjectInspector: inspects [[java.util.Map ]]
123+ * Struct.StructObjectInspector: inspects java array, [[java.util.List ]] and
60124 * even a normal java object (POJO)
61125 * UnionObjectInspector: (tag: Int, object data) (TODO: not supported by SparkSQL yet)
62126 *
@@ -72,12 +136,41 @@ import scala.collection.JavaConversions._
72136 }}}
73137 * Hive provides 3 built-in constant object inspectors:
74138 * Primitive Object Inspectors:
75- * [[WritableConstantIntObjectInspector ]],
76- * [[WritableConstantDateObjectInspector ]] etc.
139+ * WritableConstantStringObjectInspector
140+ * WritableConstantHiveVarcharObjectInspector
141+ * WritableConstantHiveDecimalObjectInspector
142+ * WritableConstantTimestampObjectInspector
143+ * WritableConstantIntObjectInspector
144+ * WritableConstantDoubleObjectInspector
145+ * WritableConstantBooleanObjectInspector
146+ * WritableConstantLongObjectInspector
147+ * WritableConstantFloatObjectInspector
148+ * WritableConstantShortObjectInspector
149+ * WritableConstantByteObjectInspector
150+ * WritableConstantBinaryObjectInspector
151+ * WritableConstantDateObjectInspector
77152 * Map Object Inspector:
78- * [[ StandardConstantMapObjectInspector ]]
153+ * StandardConstantMapObjectInspector
79154 * List Object Inspector:
80- * [[StandardConstantListObjectInspector ]]
155+ * StandardConstantListObjectInspector]]
156+ * Struct Object Inspector: Hive doesn't provide the built-in constant object inspector for Struct
157+ * Union Object Inspector: Hive doesn't provide the built-in constant object inspector for Union
158+ *
159+ *
160+ * 3. This trait facilitates:
161+ * Data Unwrapping: Hive Data => Catalyst Data (unwrap)
162+ * Data Wrapping: Catalyst Data => Hive Data (wrap)
163+ * Binding the Object Inspector for Catalyst Data (toInspector)
164+ * Retrieving the Catalyst Data Type from Object Inspector (inspectorToDataType)
165+ *
166+ *
167+ * 4. Future Improvement (TODO)
168+ * This implementation is quite ugly and inefficient:
169+ * a. Pattern matching in runtime
170+ * b. Small objects creation in catalyst data => writable
171+ * c. Unnecessary unwrap / wrap for nested UDF invoking:
172+ * e.g. date_add(printf("%s-%s-%s", a,b,c), 3)
173+ * We don't need to unwrap the data for printf and wrap it again and passes in data_add
81174 */
82175private [hive] trait HiveInspectors {
83176
@@ -145,17 +238,6 @@ private[hive] trait HiveInspectors {
145238 * extract writable from `data` and then get the catalyst type from the writable
146239 * Extract the java object directly from the object inspector
147240 *
148- * The output data (catalyst type) can be one of following:
149- * PrimitiveObjectInspector =>
150- * [[java.sql.Date ]]
151- * [[java.sql.Timestamp ]]
152- * [[org.apache.spark.sql.catalyst.types.decimal.Decimal ]]
153- * [[Array[Byte ]]
154- * java.lang.xx
155- * MapObjectInspector => [[scala.collection.immutable.Map ]]
156- * ListObjectInspector => [[scala.collection.immutable.Seq ]]
157- * StructObjectInspector => GenericRow
158- *
159241 * NOTICE: the complex data type requires recursive unwrapping.
160242 */
161243 def unwrap (data : Any , oi : ObjectInspector ): Any = oi match {
@@ -296,21 +378,10 @@ private[hive] trait HiveInspectors {
296378 * toInspector: Expression => ObjectInspector
297379 *
298380 * Strictly follows the following order in wrapping (constant OI has the higher priority):
299- * Constant object inspector => return the bundled value of Constant object inspector
300- * Check whether the `data` is null => return null if true
301- * If object inspector prefers writable => return a Writable for the given data `a`
302- * Map the catalyst data to the boxed java primitive
303- *
304- * The input data structure (catalyst type) could be one of following:
305- * PrimitiveObjectInspector =>
306- * [[java.sql.Date ]]
307- * [[java.sql.Timestamp ]]
308- * [[org.apache.spark.sql.catalyst.types.decimal.Decimal ]]
309- * [[Array[Byte ]]
310- * java.lang.xx
311- * MapObjectInspector => [[scala.collection.immutable.Map ]]
312- * ListObjectInspector => [[scala.collection.immutable.Seq ]]
313- * StructObjectInspector => Row/[[scala.collection.immutable.Seq ]]
381+ * Constant object inspector => return the bundled value of Constant object inspector
382+ * Check whether the `a` is null => return null if true
383+ * If object inspector prefers writable object => return a Writable for the given data `a`
384+ * Map the catalyst data to the boxed java primitive
314385 *
315386 * NOTICE: the complex data type requires recursive wrapping.
316387 */
@@ -405,17 +476,7 @@ private[hive] trait HiveInspectors {
405476 /**
406477 * @param dataType Catalyst data type
407478 * @return Hive java object inspector (recursively), not the Writable ObjectInspector
408- * The catalyst native data type only will be one of following:
409- * Primitive =>
410- * [[java.sql.Date ]]
411- * [[java.sql.Timestamp ]]
412- * [[org.apache.spark.sql.catalyst.types.decimal.Decimal ]]
413- * [[Array[Byte ]]
414- * [[java.lang.Integer ]], [[java.lang.Long ]] etc.
415- * Map => [[scala.collection.immutable.Map ]]
416- * List => [[scala.collection.immutable.Seq ]]
417- * Struct => Row/[[scala.collection.immutable.Seq ]]
418- * We can easily map to the Hive built-in object inspector according to the data type.
479+ * We can easily map to the Hive built-in object inspector according to the data type.
419480 */
420481 def toInspector (dataType : DataType ): ObjectInspector = dataType match {
421482 case ArrayType (tpe, _) =>
@@ -497,7 +558,7 @@ private[hive] trait HiveInspectors {
497558 })
498559 ObjectInspectorFactory .getStandardConstantMapObjectInspector(keyOI, valueOI, map)
499560 }
500- // We will enumerate all of the possible constant expressions, exception raised if we missed
561+ // We will enumerate all of the possible constant expressions, throw exception if we missed
501562 case Literal (_, dt) => sys.error(s " Hive doesn't support the constant type [ $dt]. " )
502563 // ideally, we don't test the foldable here(but in optimizer), however, some of the
503564 // Hive UDF / UDAF requires its argument to be constant objectinspector, we do it eagerly.
0 commit comments