Github user BryanCutler commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18655#discussion_r128061312
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
    @@ -55,145 +51,55 @@ private[sql] class ArrowPayload private[arrow] 
(payload: Array[Byte]) extends Se
       def asPythonSerializable: Array[Byte] = payload
     }
     
    -private[sql] object ArrowPayload {
    -
    -  /**
    -   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
    -   */
    -  def apply(
    -      batch: ArrowRecordBatch,
    -      schema: StructType,
    -      allocator: BufferAllocator): ArrowPayload = {
    -    new ArrowPayload(ArrowConverters.batchToByteArray(batch, schema, 
allocator))
    -  }
    -}
    -
     private[sql] object ArrowConverters {
     
       /**
    -   * Map a Spark DataType to ArrowType.
    -   */
    -  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
    -    dataType match {
    -      case BooleanType => ArrowType.Bool.INSTANCE
    -      case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
    -      case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
    -      case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
    -      case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
    -      case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
    -      case ByteType => new ArrowType.Int(8, true)
    -      case StringType => ArrowType.Utf8.INSTANCE
    -      case BinaryType => ArrowType.Binary.INSTANCE
    -      case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
    -    }
    -  }
    -
    -  /**
    -   * Convert a Spark Dataset schema to Arrow schema.
    -   */
    -  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
    -    val arrowFields = schema.fields.map { f =>
    -      new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
    -    }
    -    new Schema(arrowFields.toList.asJava)
    -  }
    -
    -  /**
        * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
        * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
        */
       private[sql] def toPayloadIterator(
           rowIter: Iterator[InternalRow],
           schema: StructType,
    -      maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
    -    new Iterator[ArrowPayload] {
    -      private val _allocator = new RootAllocator(Long.MaxValue)
    -      private var _nextPayload = if (rowIter.nonEmpty) convert() else null
    +      maxRecordsPerBatch: Int,
    +      context: TaskContext): Iterator[ArrowPayload] = {
     
    -      override def hasNext: Boolean = _nextPayload != null
    +    val arrowSchema = ArrowUtils.toArrowSchema(schema)
    +    val allocator =
    +      ArrowUtils.rootAllocator.newChildAllocator("toPayloadIterator", 0, 
Long.MaxValue)
     
    -      override def next(): ArrowPayload = {
    -        val obj = _nextPayload
    -        if (hasNext) {
    -          if (rowIter.hasNext) {
    -            _nextPayload = convert()
    -          } else {
    -            _allocator.close()
    -            _nextPayload = null
    -          }
    -        }
    -        obj
    -      }
    -
    -      private def convert(): ArrowPayload = {
    -        val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator, maxRecordsPerBatch)
    -        ArrowPayload(batch, schema, _allocator)
    -      }
    -    }
    -  }
    -
    -  /**
    -   * Iterate over InternalRows and write to an ArrowRecordBatch, stopping 
when rowIter is consumed
    -   * or the number of records in the batch equals maxRecordsInBatch.  If 
maxRecordsPerBatch is 0,
    -   * then rowIter will be fully consumed.
    -   */
    -  private def internalRowIterToArrowBatch(
    -      rowIter: Iterator[InternalRow],
    -      schema: StructType,
    -      allocator: BufferAllocator,
    -      maxRecordsPerBatch: Int = 0): ArrowRecordBatch = {
    -
    -    val columnWriters = schema.fields.zipWithIndex.map { case (field, 
ordinal) =>
    -      ColumnWriter(field.dataType, ordinal, allocator).init()
    -    }
    +    val root = VectorSchemaRoot.create(arrowSchema, allocator)
    +    val arrowWriter = ArrowWriter.create(root)
     
    -    val writerLength = columnWriters.length
    -    var recordsInBatch = 0
    -    while (rowIter.hasNext && (maxRecordsPerBatch <= 0 || recordsInBatch < 
maxRecordsPerBatch)) {
    -      val row = rowIter.next()
    -      var i = 0
    -      while (i < writerLength) {
    -        columnWriters(i).write(row)
    -        i += 1
    -      }
    -      recordsInBatch += 1
    +    context.addTaskCompletionListener { _ =>
    +      root.close()
    +      allocator.close()
    --- End diff --
    
    It seems a little odd to me to tie an iterator to a TaskContext, why not 
just close resources as soon as the row iterator is consumed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to