-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-54446][SQL][ML][CONNECT] FPGrowth supports local filesystem with Arrow file format #53232
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 7 commits
d93588d
9474f69
07f67e6
536b403
4dcf366
e09ece3
76bc0a8
c7c2db5
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 |
|---|---|---|
|
|
@@ -46,7 +46,8 @@ import org.apache.spark.ml.feature.RFormulaModel | |
| import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, Matrix, SparseMatrix, SparseVector, Vector} | ||
| import org.apache.spark.ml.param.{ParamPair, Params} | ||
| import org.apache.spark.ml.tuning.ValidatorParams | ||
| import org.apache.spark.sql.{SparkSession, SQLContext} | ||
| import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext} | ||
| import org.apache.spark.sql.execution.arrow.ArrowFileReadWrite | ||
| import org.apache.spark.util.{Utils, VersionUtils} | ||
|
|
||
| /** | ||
|
|
@@ -1142,4 +1143,31 @@ private[spark] object ReadWriteUtils { | |
| spark.read.parquet(path).as[T].collect() | ||
| } | ||
| } | ||
|
|
||
| def saveDataFrame(path: String, df: DataFrame): Unit = { | ||
| if (localSavingModeState.get()) { | ||
| val filePath = Paths.get(path) | ||
| Files.createDirectories(filePath.getParent) | ||
|
|
||
| df match { | ||
| case d: org.apache.spark.sql.classic.DataFrame => | ||
| ArrowFileReadWrite.save(d, path) | ||
| case _ => throw new UnsupportedOperationException("Unsupported dataframe type") | ||
| } | ||
| } else { | ||
| df.write.parquet(path) | ||
| } | ||
| } | ||
|
|
||
| def loadDataFrame(path: String, spark: SparkSession): DataFrame = { | ||
| if (localSavingModeState.get()) { | ||
| spark match { | ||
| case s: org.apache.spark.sql.classic.SparkSession => | ||
| ArrowFileReadWrite.load(s, path) | ||
| case _ => throw new UnsupportedOperationException("Unsupported session type") | ||
zhengruifeng marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| } else { | ||
| spark.read.parquet(path) | ||
| } | ||
| } | ||
|
Comment on lines
1147
to
1173
Contributor
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. So if we have
Contributor
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. hi @holdenk , as @WeichenXu123 explained #53150 (comment), this is a runtime temporary file in spark connect server side, and will be cleaned after session close.
Contributor
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.
Member
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. Hmm, even it is just a temporary session file, is there any reason not to use Parquet but Arrow file format?
Contributor
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. we can read/write parquet with arrow, but it requires a new dependency otherwise, I am not sure whether we have utils to read/write parquet.
Contributor
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. At the end we need the in-memory data to be in arrow format, so using arrow file is more efficient. |
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution.arrow | ||
|
|
||
| import java.io.{ByteArrayOutputStream, FileOutputStream} | ||
| import java.nio.channels.Channels | ||
| import java.nio.file.Files | ||
| import java.nio.file.Paths | ||
|
|
||
| import scala.jdk.CollectionConverters._ | ||
|
|
||
| import org.apache.arrow.vector._ | ||
| import org.apache.arrow.vector.ipc.{ArrowFileReader, ArrowFileWriter, WriteChannel} | ||
| import org.apache.arrow.vector.ipc.message.MessageSerializer | ||
| import org.apache.arrow.vector.types.pojo.Schema | ||
|
|
||
| import org.apache.spark.sql.classic.{DataFrame, SparkSession} | ||
| import org.apache.spark.sql.util.ArrowUtils | ||
|
|
||
| private[sql] class SparkArrowFileWriter( | ||
| arrowSchema: Schema, | ||
zhengruifeng marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| path: String) extends AutoCloseable { | ||
| private val allocator = | ||
| ArrowUtils.rootAllocator.newChildAllocator( | ||
| s"to${this.getClass.getSimpleName}", 0, Long.MaxValue) | ||
|
|
||
| protected val root = VectorSchemaRoot.create(arrowSchema, allocator) | ||
| protected val loader = new VectorLoader(root) | ||
| protected val arrowWriter = ArrowWriter.create(root) | ||
zhengruifeng marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
| protected val fileWriter = | ||
| new ArrowFileWriter(root, null, Channels.newChannel(new FileOutputStream(path))) | ||
|
|
||
| override def close(): Unit = { | ||
| root.close() | ||
| allocator.close() | ||
| fileWriter.close() | ||
| } | ||
|
|
||
| def write(batchBytesIter: Iterator[Array[Byte]]): Unit = { | ||
zhengruifeng marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| fileWriter.start() | ||
| while (batchBytesIter.hasNext) { | ||
| val batchBytes = batchBytesIter.next() | ||
| val batch = ArrowConverters.loadBatch(batchBytes, allocator) | ||
|
Contributor
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. The |
||
| loader.load(batch) | ||
| fileWriter.writeBatch() | ||
| } | ||
| fileWriter.close() | ||
| } | ||
| } | ||
|
|
||
| private[sql] class SparkArrowFileReader(path: String) extends AutoCloseable { | ||
| private val allocator = | ||
| ArrowUtils.rootAllocator.newChildAllocator( | ||
| s"to${this.getClass.getSimpleName}", 0, Long.MaxValue) | ||
|
|
||
| protected val fileReader = | ||
| new ArrowFileReader(Files.newByteChannel(Paths.get(path)), allocator) | ||
|
|
||
| override def close(): Unit = { | ||
| allocator.close() | ||
| fileReader.close() | ||
| } | ||
|
|
||
| val schema: Schema = fileReader.getVectorSchemaRoot.getSchema | ||
|
|
||
| def read(): Iterator[Array[Byte]] = { | ||
| fileReader.getRecordBlocks.iterator().asScala.map { block => | ||
| fileReader.loadRecordBatch(block) | ||
| val root = fileReader.getVectorSchemaRoot | ||
| val unloader = new VectorUnloader(root) | ||
| val batch = unloader.getRecordBatch | ||
| val out = new ByteArrayOutputStream() | ||
| val writeChannel = new WriteChannel(Channels.newChannel(out)) | ||
| MessageSerializer.serialize(writeChannel, batch) | ||
| out.toByteArray | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private[spark] object ArrowFileReadWrite { | ||
| def save(df: DataFrame, path: String): Unit = { | ||
| val maxRecordsPerBatch = df.sparkSession.sessionState.conf.arrowMaxRecordsPerBatch | ||
| val rdd = df.toArrowBatchRdd(maxRecordsPerBatch, "UTC", true, false) | ||
| val arrowSchema = ArrowUtils.toArrowSchema(df.schema, "UTC", true, false) | ||
| val writer = new SparkArrowFileWriter(arrowSchema, path) | ||
| writer.write(rdd.toLocalIterator) | ||
|
Member
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. Instead, can we call
Contributor
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. we can make best of the |
||
| } | ||
|
|
||
| def load(spark: SparkSession, path: String): DataFrame = { | ||
| val reader = new SparkArrowFileReader(path) | ||
| val schema = ArrowUtils.fromArrowSchema(reader.schema) | ||
| ArrowConverters.toDataFrame(reader.read(), schema, spark, "UTC", true, false) | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.