-
Notifications
You must be signed in to change notification settings - Fork 2
Arrow integration #13
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,41 @@ | ||
| import pyspark | ||
| import timeit | ||
| import random | ||
| from pyspark.sql import SparkSession | ||
|
|
||
| numPartition = 20 | ||
|
|
||
| def time(df, repeat, number): | ||
| print("toPandas with arrow") | ||
| print(timeit.repeat('df.toPandas(True)', repeat=repeat, number=number, globals={'df': df})) | ||
|
|
||
| print("toPandas without arrow") | ||
| print(timeit.repeat('df.toPandas(False)', repeat=repeat, number=number, globals={'df': df})) | ||
|
|
||
| def long(): | ||
| return random.randint(0, 10000) | ||
|
|
||
| def double(): | ||
| return random.random() | ||
|
|
||
| def genDataLocal(spark, size, columns): | ||
| data = [list([fn() for fn in columns]) for x in range(0, size)] | ||
| df = spark.createDataFrame(data) | ||
| return df | ||
|
|
||
| def genData(spark, size, columns): | ||
| rdd = (spark.sparkContext | ||
| .parallelize(range(0,numPartition), numPartition) | ||
| .flatMap(lambda index: [list([fn() for fn in columns]) for x in range(0, int(size / numPartition))])) | ||
| df = spark.createDataFrame(rdd) | ||
| return df | ||
|
|
||
| if __name__ == "__main__": | ||
| spark = SparkSession.builder.appName("ArrowBenchmark").getOrCreate() | ||
| df = genData(spark, 1000 * 1000, [long, double]) | ||
| df.cache() | ||
| df.count() | ||
|
|
||
| time(df, 10, 1) | ||
|
|
||
| df.unpersist() |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ import scala.util.control.NonFatal | |
|
|
||
| import io.netty.buffer.ArrowBuf | ||
| import org.apache.arrow.memory.RootAllocator | ||
| import org.apache.arrow.vector.BitVector | ||
| import org.apache.arrow.vector.file.ArrowWriter | ||
| import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch} | ||
| import org.apache.arrow.vector.types.FloatingPointPrecision | ||
|
|
@@ -60,7 +61,6 @@ import org.apache.spark.storage.StorageLevel | |
| import org.apache.spark.unsafe.types.CalendarInterval | ||
| import org.apache.spark.util.Utils | ||
|
|
||
|
|
||
| private[sql] object Dataset { | ||
| def apply[T: Encoder](sparkSession: SparkSession, logicalPlan: LogicalPlan): Dataset[T] = { | ||
| new Dataset(sparkSession, logicalPlan, implicitly[Encoder[T]]) | ||
|
|
@@ -2426,6 +2426,29 @@ class Dataset[T] private[sql]( | |
| Math.ceil(numOfRows / 64.0).toInt * 8 | ||
| } | ||
|
|
||
| private def fillArrow(buf: ArrowBuf, dataType: DataType): Unit = { | ||
| dataType match { | ||
| case NullType => | ||
| case BooleanType => | ||
| buf.writeBoolean(false) | ||
| case ShortType => | ||
| buf.writeShort(0) | ||
| case IntegerType => | ||
| buf.writeInt(0) | ||
| case LongType => | ||
| buf.writeLong(0L) | ||
| case FloatType => | ||
| buf.writeFloat(0f) | ||
| case DoubleType => | ||
| buf.writeDouble(0d) | ||
| case ByteType => | ||
| buf.writeByte(0) | ||
| case _ => | ||
| throw new UnsupportedOperationException( | ||
| s"Unsupported data type ${dataType.simpleString}") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Get an entry from the InternalRow, and then set to ArrowBuf. | ||
| * Note: No Null check for the entry. | ||
|
|
@@ -2466,20 +2489,26 @@ class Dataset[T] private[sql]( | |
|
|
||
| field.dataType match { | ||
| case IntegerType | LongType | DoubleType | FloatType | BooleanType | ByteType => | ||
| val validity = allocator.buffer(numBytesOfBitmap(numOfRows)) | ||
| val validityVector = new BitVector("validity", allocator) | ||
| val validityMutator = validityVector.getMutator() | ||
| validityVector.allocateNew(numOfRows) | ||
| validityMutator.setValueCount(numOfRows) | ||
| val buf = allocator.buffer(numOfRows * field.dataType.defaultSize) | ||
| var nullCount = 0 | ||
| rows.foreach { row => | ||
| rows.zipWithIndex.foreach { case (row, index) => | ||
| if (row.isNullAt(ordinal)) { | ||
| nullCount += 1 | ||
| validityMutator.set(index, 0) | ||
| fillArrow(buf, field.dataType) | ||
|
Owner
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just to clarify, so the buffer must contain values at each "null" position? Is the case for
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, here are the specs of the arrow layout: |
||
| } else { | ||
| validityMutator.set(index, 1) | ||
| getAndSetToArrow(row, buf, field.dataType, ordinal) | ||
| } | ||
| } | ||
|
|
||
| val fieldNode = new ArrowFieldNode(numOfRows, nullCount) | ||
|
|
||
| (Array(validity, buf), Array(fieldNode)) | ||
| (Array(validityVector.getBuffer, buf), Array(fieldNode)) | ||
|
|
||
| case StringType => | ||
| val validityOffset = allocator.buffer(numBytesOfBitmap(numOfRows)) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it would be better just use a while loop here since
zipWithIndexwill iterate and copy the items in an arrayThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point. I am going to refactor this bit of code later to be more efficient. Do you want to wait until that is done or do you want to merge this first?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'll leave this open for a day or so, so Xusen can have a look since he wrote the conversion code. If you want to update go ahead or I can before I merge, no biggie. I also would like to have
fillWithArrowandgetAndSetToArrowuse the same data type cases to avoid duplication, but I can do that later.