@@ -193,6 +193,31 @@ private[parquet] object ParquetTypesConverter {
193193 s " Unsupported parquet datatype $parquetType" )
194194 }
195195
196+ /**
197+ * Converts a given Parquet `Type` into the corresponding
198+ * [[org.apache.spark.sql.catalyst.types.DataType ]].
199+ *
200+ * Note that we apply the following conversion rules:
201+ * <ul>
202+ * <li> Primitive types are converter to the corresponding primitive type.</li>
203+ * <li> Group types that have a single field with repetition `REPEATED` or themselves
204+ * have repetition level `REPEATED` are converted to an [[ArrayType ]] with the
205+ * corresponding field type (possibly primitive) as element type.</li>
206+ * <li> Other group types are converted as follows:<ul>
207+ * <li> If they have a single field, they are converted into a [[StructType ]] with
208+ * the corresponding field type.</li>
209+ * <li> If they have more than one field and repetition level `REPEATED` they are
210+ * converted into an [[ArrayType ]] with the corresponding [[StructType ]] as complex
211+ * element type.</li>
212+ * <li> Otherwise they are converted into a [[StructType ]] with the corresponding
213+ * field types.</li></ul></li>
214+ * </ul>
215+ * Note that fields are determined to be `nullable` if and only if their Parquet repetition
216+ * level is not `REQUIRED`.
217+ *
218+ * @param parquetType The type to convert.
219+ * @return The corresponding Catalyst type.
220+ */
196221 def toDataType (parquetType : ParquetType ): DataType = {
197222 if (parquetType.isPrimitive) {
198223 toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName)
@@ -215,7 +240,9 @@ private[parquet] object ParquetTypesConverter {
215240 case _ => { // everything else nested becomes a Struct, unless it has a single repeated field
216241 // in which case it becomes an array (this should correspond to the inverse operation of
217242 // parquet.schema.ConversionPatterns.listType)
218- if (groupType.getFieldCount == 1 && groupType.getFields.apply(0 ).getRepetition == Repetition .REPEATED ) {
243+ if (groupType.getFieldCount == 1 &&
244+ (groupType.getFields.apply(0 ).getRepetition == Repetition .REPEATED ||
245+ groupType.getRepetition == Repetition .REPEATED )) {
219246 val elementType = toDataType(groupType.getFields.apply(0 ))
220247 new ArrayType (elementType)
221248 } else {
@@ -225,9 +252,10 @@ private[parquet] object ParquetTypesConverter {
225252 ptype.getName,
226253 toDataType(ptype),
227254 ptype.getRepetition != Repetition .REQUIRED ))
228- if (groupType.getFieldCount == 1 ) { // single field, either optional or required
255+
256+ if (groupType.getFieldCount == 1 ) {
229257 new StructType (fields)
230- } else { // multi field repeated group, which we map into an array of structs
258+ } else {
231259 if (parquetType.getRepetition == Repetition .REPEATED ) {
232260 new ArrayType (StructType (fields))
233261 } else {
@@ -240,6 +268,14 @@ private[parquet] object ParquetTypesConverter {
240268 }
241269 }
242270
271+ /**
272+ * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType ]] return
273+ * the name of the corresponding Parquet primitive type or None if the given type
274+ * is not primitive.
275+ *
276+ * @param ctype The type to convert
277+ * @return The name of the corresponding Parquet primitive type
278+ */
243279 def fromPrimitiveDataType (ctype : DataType ): Option [ParquetPrimitiveTypeName ] = ctype match {
244280 case StringType => Some (ParquetPrimitiveTypeName .BINARY )
245281 case BooleanType => Some (ParquetPrimitiveTypeName .BOOLEAN )
@@ -251,6 +287,41 @@ private[parquet] object ParquetTypesConverter {
251287 case _ => None
252288 }
253289
290+ /**
291+ * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType ]] into
292+ * the corrponsing Parquet `Type`.
293+ *
294+ * The conversion follows the rules below:
295+ * <ul>
296+ * <li> Primitive types are converted into Parquet's primitive types.</li>
297+ * <li> [[org.apache.spark.sql.catalyst.types.StructType ]]s are converted
298+ * into Parquet's `GroupType` with the corresponding field types.</li>
299+ * <li> [[org.apache.spark.sql.catalyst.types.ArrayType ]]s are handled as follows:<ul>
300+ * <li> If their element is complex, that is of type
301+ * [[org.apache.spark.sql.catalyst.types.StructType ]], they are converted
302+ * into a `GroupType` with the corresponding field types of the struct and
303+ * original type of the `GroupType` is set to `LIST`.</li>
304+ * <li> Otherwise, that is they contain a primitive they are converted into a `GroupType`
305+ * that is also a list but has only a single field of the type corresponding to
306+ * the element type.</li></ul></li>
307+ * </ul>
308+ * Parquet's repetition level is set according to the following rule:
309+ * <ul>
310+ * <li> If the call to `fromDataType` is recursive inside an enclosing `ArrayType`, then
311+ * the repetition level is set to `REPEATED`.</li>
312+ * <li> Otherwise, if the attribute whose type is converted is `nullable`, the Parquet
313+ * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.</li>
314+ * </ul>
315+ * The single expection to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType ]]
316+ * that contains a [[org.apache.spark.sql.catalyst.types.StructType ]], whose repetition level
317+ * is always set to `REPEATED`.
318+ *
319+ @param ctype The type to convert.
320+ * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute ]] whose type is converted
321+ * @param nullable When true indicates that the attribute is nullable
322+ * @param inArray When true indicates that this is a nested attribute inside an array.
323+ * @return The corresponding Parquet type.
324+ */
254325 def fromDataType (
255326 ctype : DataType ,
256327 name : String ,
@@ -271,8 +342,9 @@ private[parquet] object ParquetTypesConverter {
271342 elementType match {
272343 case StructType (fields) => { // first case: array of structs
273344 val parquetFieldTypes = fields.map(
274- f => fromDataType(f.dataType, f.name, f.nullable, false ))
275- new ParquetGroupType (repetition, name, ParquetOriginalType .LIST , parquetFieldTypes)
345+ f => fromDataType(f.dataType, f.name, f.nullable, inArray = false ))
346+ assert(fields.size > 1 , " Found struct inside array with a single field.. error parsin Catalyst schema" )
347+ new ParquetGroupType (Repetition .REPEATED , name, ParquetOriginalType .LIST , parquetFieldTypes)
276348 // ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes)
277349 }
278350 case _ => { // second case: array of primitive types
@@ -288,7 +360,7 @@ private[parquet] object ParquetTypesConverter {
288360 // TODO: test structs inside arrays
289361 case StructType (structFields) => {
290362 val fields = structFields.map {
291- field => fromDataType(field.dataType, field.name, field.nullable)
363+ field => fromDataType(field.dataType, field.name, field.nullable, inArray = false )
292364 }
293365 new ParquetGroupType (repetition, name, fields)
294366 }
0 commit comments