Skip to content

Commit a099d3e

Browse files
committed
More comments
1 parent 0cc1b37 commit a099d3e

File tree

1 file changed

+8
-2
lines changed

1 file changed

+8
-2
lines changed

sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -313,6 +313,9 @@ private[parquet] class CatalystRowConverter(
313313

314314
override def end(): Unit = updater.set(currentArray)
315315

316+
// NOTE: We can't reuse the mutable Map here and must instantiate a new `ArrayBuffer` for the
317+
// next value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored
318+
// in row cells.
316319
override def start(): Unit = currentArray = ArrayBuffer.empty[Any]
317320

318321
// scalastyle:off
@@ -334,8 +337,8 @@ private[parquet] class CatalystRowConverter(
334337
* @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules
335338
*/
336339
// scalastyle:on
337-
private def isElementType(repeatedType: Type, elementType: DataType): Boolean = {
338-
(repeatedType, elementType) match {
340+
private def isElementType(parquetRepeatedType: Type, catalystElementType: DataType): Boolean = {
341+
(parquetRepeatedType, catalystElementType) match {
339342
case (t: PrimitiveType, _) => true
340343
case (t: GroupType, _) if t.getFieldCount > 1 => true
341344
case (t: GroupType, StructType(Array(f))) if f.name == t.getFieldName(0) => true
@@ -383,6 +386,9 @@ private[parquet] class CatalystRowConverter(
383386

384387
override def end(): Unit = updater.set(currentMap)
385388

389+
// NOTE: We can't reuse the mutable Map here and must instantiate a new `Map` for the next
390+
// value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored in row
391+
// cells.
386392
override def start(): Unit = currentMap = mutable.Map.empty[Any, Any]
387393

388394
/** Parquet converter for key-value pairs within the map. */

0 commit comments

Comments
 (0)