-
Notifications
You must be signed in to change notification settings - Fork 935
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[KYUUBI #5377] Spark engine query result save to file
### _Why are the changes needed?_ close #5377 ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [ ] [Run test](https://kyuubi.readthedocs.io/en/master/contributing/code/testing.html#running-tests) locally before make a pull request ### _Was this patch authored or co-authored using generative AI tooling?_ NO Closes #5591 from lsm1/branch-kyuubi-5377. Closes #5377 9d1a18c [senmiaoliu] ignore empty file 3c70a1e [LSM] fix doc 73d3c3a [senmiaoliu] fix style and add some comment 80e1f0d [senmiaoliu] Close orc fetchOrcStatement and remove result save file when ExecuteStatement close 42634a1 [senmiaoliu] fix style 979125d [senmiaoliu] fix style 1dc07a5 [senmiaoliu] spark engine save into hdfs file Lead-authored-by: senmiaoliu <[email protected]> Co-authored-by: LSM <[email protected]> Signed-off-by: Fu Chen <[email protected]>
- Loading branch information
Showing
7 changed files
with
271 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
151 changes: 151 additions & 0 deletions
151
...ql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/FetchOrcStatement.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,151 @@ | ||
/* | ||
* 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.kyuubi.engine.spark.operation | ||
|
||
import scala.Array._ | ||
import scala.collection.mutable.ListBuffer | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{LocatedFileStatus, Path} | ||
import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} | ||
import org.apache.hadoop.mapreduce.lib.input.FileSplit | ||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
import org.apache.orc.mapred.OrcStruct | ||
import org.apache.orc.mapreduce.OrcInputFormat | ||
import org.apache.spark.sql.{Row, SparkSession} | ||
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | ||
import org.apache.spark.sql.catalyst.expressions.AttributeReference | ||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
import org.apache.spark.sql.execution.datasources.RecordReaderIterator | ||
import org.apache.spark.sql.execution.datasources.orc.OrcDeserializer | ||
import org.apache.spark.sql.types.StructType | ||
|
||
import org.apache.kyuubi.KyuubiException | ||
import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.SPARK_ENGINE_RUNTIME_VERSION | ||
import org.apache.kyuubi.operation.{FetchIterator, IterableFetchIterator} | ||
import org.apache.kyuubi.util.reflect.DynConstructors | ||
|
||
class FetchOrcStatement(spark: SparkSession) { | ||
|
||
var orcIter: OrcFileIterator = _ | ||
def getIterator(path: String, orcSchema: StructType): FetchIterator[Row] = { | ||
val conf = spark.sparkContext.hadoopConfiguration | ||
val savePath = new Path(path) | ||
val fsIterator = savePath.getFileSystem(conf).listFiles(savePath, false) | ||
val list = new ListBuffer[LocatedFileStatus] | ||
while (fsIterator.hasNext) { | ||
val file = fsIterator.next() | ||
if (file.getPath.getName.endsWith(".orc") && file.getLen > 0) { | ||
list += file | ||
} | ||
} | ||
val toRowConverter: InternalRow => Row = { | ||
CatalystTypeConverters.createToScalaConverter(orcSchema) | ||
.asInstanceOf[InternalRow => Row] | ||
} | ||
val colId = range(0, orcSchema.size) | ||
val fullSchema = orcSchema.map(f => | ||
AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) | ||
val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) | ||
val deserializer = getOrcDeserializer(orcSchema, colId) | ||
orcIter = new OrcFileIterator(list) | ||
val iterRow = orcIter.map(value => | ||
unsafeProjection(deserializer.deserialize(value))) | ||
.map(value => toRowConverter(value)) | ||
new IterableFetchIterator[Row](iterRow.toIterable) | ||
} | ||
|
||
def close(): Unit = { | ||
orcIter.close() | ||
} | ||
|
||
private def getOrcDeserializer(orcSchema: StructType, colId: Array[Int]): OrcDeserializer = { | ||
try { | ||
if (SPARK_ENGINE_RUNTIME_VERSION >= "3.2") { | ||
// SPARK-34535 changed the constructor signature of OrcDeserializer | ||
DynConstructors.builder() | ||
.impl(classOf[OrcDeserializer], classOf[StructType], classOf[Array[Int]]) | ||
.build[OrcDeserializer]() | ||
.newInstance( | ||
orcSchema, | ||
colId) | ||
} else { | ||
DynConstructors.builder() | ||
.impl( | ||
classOf[OrcDeserializer], | ||
classOf[StructType], | ||
classOf[StructType], | ||
classOf[Array[Int]]) | ||
.build[OrcDeserializer]() | ||
.newInstance( | ||
new StructType, | ||
orcSchema, | ||
colId) | ||
} | ||
} catch { | ||
case e: Throwable => | ||
throw new KyuubiException("Failed to create OrcDeserializer", e) | ||
} | ||
} | ||
} | ||
|
||
class OrcFileIterator(fileList: ListBuffer[LocatedFileStatus]) extends Iterator[OrcStruct] { | ||
|
||
private val iters = fileList.map(x => getOrcFileIterator(x)) | ||
|
||
var idx = 0 | ||
|
||
override def hasNext: Boolean = { | ||
val hasNext = iters(idx).hasNext | ||
if (!hasNext) { | ||
iters(idx).close() | ||
idx += 1 | ||
// skip empty file | ||
while (idx < iters.size) { | ||
if (iters(idx).hasNext) { | ||
return true | ||
} else { | ||
iters(idx).close() | ||
idx = idx + 1 | ||
} | ||
} | ||
} | ||
hasNext | ||
} | ||
|
||
override def next(): OrcStruct = { | ||
iters(idx).next() | ||
} | ||
|
||
def close(): Unit = { | ||
iters.foreach(_.close()) | ||
} | ||
|
||
private def getOrcFileIterator(file: LocatedFileStatus): RecordReaderIterator[OrcStruct] = { | ||
val orcRecordReader = { | ||
val split = | ||
new FileSplit(file.getPath, 0, file.getLen, Array.empty[String]) | ||
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) | ||
val hadoopAttemptContext = | ||
new TaskAttemptContextImpl(new Configuration(), attemptId) | ||
val oif = new OrcInputFormat[OrcStruct] | ||
oif.createRecordReader(split, hadoopAttemptContext) | ||
} | ||
new RecordReaderIterator[OrcStruct](orcRecordReader) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.