Skip to content

Commit

Permalink
move getLocalFilesNode logic to transformer
Browse files Browse the repository at this point in the history
  • Loading branch information
liujiayi771 committed Nov 10, 2023
1 parent 4a72871 commit 4460e2f
Show file tree
Hide file tree
Showing 16 changed files with 284 additions and 211 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import io.glutenproject.backendsapi.IteratorApi
import io.glutenproject.execution._
import io.glutenproject.metrics.{GlutenTimeMetric, IMetrics, NativeMetrics}
import io.glutenproject.substrait.plan.PlanNode
import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder}
import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder, ReadSplit}
import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat
import io.glutenproject.utils.{LogLevelUtil, SubstraitPlanPrinterUtil}
import io.glutenproject.vectorized.{CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator, GeneralInIterator, GeneralOutIterator}
Expand All @@ -41,10 +41,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch

import java.lang.{Long => JLong}
import java.net.URI
import java.util.{ArrayList => JArrayList}
import java.util.{ArrayList => JArrayList, HashMap => JHashMap, Map => JMap}

import scala.collection.JavaConverters._
import scala.collection.mutable

class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {

Expand All @@ -53,57 +52,47 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
*
* @return
*/
override def genFilePartition(
index: Int,
partitions: Seq[InputPartition],
partitionSchemas: Seq[StructType],
fileFormats: Seq[ReadFileFormat],
wsCxt: WholeStageTransformContext): BaseGlutenPartition = {
val localFilesNodesWithLocations = partitions.indices.map(
i =>
partitions(i) match {
case p: GlutenMergeTreePartition =>
(
ExtensionTableBuilder
.makeExtensionTable(p.minParts, p.maxParts, p.database, p.table, p.tablePath),
SoftAffinityUtil.getNativeMergeTreePartitionLocations(p))
case f: FilePartition =>
val paths = new JArrayList[String]()
val starts = new JArrayList[JLong]()
val lengths = new JArrayList[JLong]()
val partitionColumns = mutable.ArrayBuffer.empty[Map[String, String]]
f.files.foreach {
file =>
paths.add(new URI(file.filePath).toASCIIString)
starts.add(JLong.valueOf(file.start))
lengths.add(JLong.valueOf(file.length))
// TODO: Support custom partition location
val partitionColumn = mutable.Map.empty[String, String]
partitionColumns.append(partitionColumn.toMap)
}
(
LocalFilesBuilder.makeLocalFiles(
f.index,
paths,
starts,
lengths,
partitionColumns.map(_.asJava).asJava,
fileFormats(i)),
SoftAffinityUtil.getFilePartitionLocations(f))
case _ =>
throw new UnsupportedOperationException(s"Unsupported input partition.")
})
wsCxt.substraitContext.initLocalFilesNodesIndex(0)
wsCxt.substraitContext.setLocalFilesNodes(localFilesNodesWithLocations.map(_._1))
val substraitPlan = wsCxt.root.toProtobuf
if (index == 0) {
logOnLevel(
GlutenConfig.getConf.substraitPlanLogLevel,
s"The substrait plan for partition $index:\n${SubstraitPlanPrinterUtil
.substraitPlanToJson(substraitPlan)}"
)
override def genReadSplit(
partition: InputPartition,
partitionSchemas: StructType,
fileFormat: ReadFileFormat): ReadSplit = {
partition match {
case p: GlutenMergeTreePartition =>
ExtensionTableBuilder
.makeExtensionTable(
p.minParts,
p.maxParts,
p.database,
p.table,
p.tablePath,
SoftAffinityUtil.getNativeMergeTreePartitionLocations(p).toList.asJava)
case f: FilePartition =>
val paths = new JArrayList[String]()
val starts = new JArrayList[JLong]()
val lengths = new JArrayList[JLong]()
val partitionColumns = new JArrayList[JMap[String, String]]
f.files.foreach {
file =>
paths.add(new URI(file.filePath).toASCIIString)
starts.add(JLong.valueOf(file.start))
lengths.add(JLong.valueOf(file.length))
// TODO: Support custom partition location
val partitionColumn = new JHashMap[String, String]()
partitionColumns.add(partitionColumn)
}
val preferredLocations =
SoftAffinityUtil.getFilePartitionLocations(paths.asScala.toArray, f.preferredLocations())
LocalFilesBuilder.makeLocalFiles(
f.index,
paths,
starts,
lengths,
partitionColumns,
fileFormat,
preferredLocations.toList.asJava)
case _ =>
throw new UnsupportedOperationException(s"Unsupported input partition.")
}
GlutenPartition(index, substraitPlan.toByteArray, localFilesNodesWithLocations.head._2)
}

/**
Expand Down Expand Up @@ -244,17 +233,28 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
override def genNativeFileScanRDD(
sparkContext: SparkContext,
wsCxt: WholeStageTransformContext,
fileFormat: ReadFileFormat,
inputPartitions: Seq[InputPartition],
readSplits: Seq[ReadSplit],
numOutputRows: SQLMetric,
numOutputBatches: SQLMetric,
scanTime: SQLMetric): RDD[ColumnarBatch] = {
val substraitPlanPartition = GlutenTimeMetric.withMillisTime {
// generate each partition of all scan exec
inputPartitions.indices.map(
i => {
genFilePartition(i, Seq(inputPartitions(i)), null, Seq(fileFormat), wsCxt)
})
readSplits.zipWithIndex.map {
case (readSplit, index) =>
wsCxt.substraitContext.initReadSplitsIndex(0)
wsCxt.substraitContext.setReadSplits(Seq(readSplit))
val substraitPlan = wsCxt.root.toProtobuf
if (index == 0) {
logOnLevel(
GlutenConfig.getConf.substraitPlanLogLevel,
s"The substrait plan for partition $index:\n${SubstraitPlanPrinterUtil
.substraitPlanToJson(substraitPlan)}"
)
}
GlutenPartition(
index,
substraitPlan.toByteArray,
readSplit.preferredLocations().asScala.toArray)
}
}(t => logInfo(s"Generating the Substrait plan took: $t ms."))

new NativeFileScanColumnarRDD(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -239,13 +239,14 @@ case class ClickHouseAppendDataExec(
starts,
lengths,
partitionColumns.map(_.asJava).asJava,
ReadFileFormat.UnknownFormat)
ReadFileFormat.UnknownFormat,
List.empty.asJava)
val insertOutputNode = InsertOutputBuilder.makeInsertOutputNode(
SnowflakeIdWorker.getInstance().nextId(),
database,
tableName,
tablePath)
dllCxt.substraitContext.setLocalFilesNodes(Seq(localFilesNode))
dllCxt.substraitContext.setReadSplits(Seq(localFilesNode))
dllCxt.substraitContext.setInsertOutputNode(insertOutputNode)
val substraitPlan = dllCxt.root.toProtobuf
logWarning(dllCxt.root.toProtobuf.toString)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,7 @@ object CHParquetReadBenchmark extends SqlBasedBenchmark with CHSqlBasedBenchmark
val nativeFileScanRDD = BackendsApiManager.getIteratorApiInstance.genNativeFileScanRDD(
spark.sparkContext,
WholeStageTransformContext(planNode, substraitContext),
fileFormat,
filePartitions,
chFileScan.getReadSplits,
numOutputRows,
numOutputVectors,
scanTime
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import io.glutenproject.backendsapi.IteratorApi
import io.glutenproject.execution._
import io.glutenproject.metrics.IMetrics
import io.glutenproject.substrait.plan.PlanNode
import io.glutenproject.substrait.rel.LocalFilesBuilder
import io.glutenproject.substrait.rel.{LocalFilesBuilder, ReadSplit}
import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat
import io.glutenproject.utils.Iterators
import io.glutenproject.vectorized._
Expand All @@ -46,11 +46,10 @@ import java.lang.{Long => JLong}
import java.net.URLDecoder
import java.nio.charset.StandardCharsets
import java.time.ZoneOffset
import java.util.{ArrayList => JArrayList}
import java.util.{ArrayList => JArrayList, HashMap => JHashMap, Map => JMap}
import java.util.concurrent.TimeUnit

import scala.collection.JavaConverters._
import scala.collection.mutable

class IteratorApiImpl extends IteratorApi with Logging {

Expand All @@ -59,71 +58,61 @@ class IteratorApiImpl extends IteratorApi with Logging {
*
* @return
*/
override def genFilePartition(
index: Int,
partitions: Seq[InputPartition],
partitionSchemas: Seq[StructType],
fileFormats: Seq[ReadFileFormat],
wsCxt: WholeStageTransformContext): BaseGlutenPartition = {

def constructSplitInfo(schema: StructType, files: Array[PartitionedFile]) = {
val paths = mutable.ArrayBuffer.empty[String]
val starts = mutable.ArrayBuffer.empty[JLong]
val lengths = mutable.ArrayBuffer.empty[JLong]
val partitionColumns = mutable.ArrayBuffer.empty[Map[String, String]]
files.foreach {
file =>
paths.append(URLDecoder.decode(file.filePath.toString, StandardCharsets.UTF_8.name()))
starts.append(JLong.valueOf(file.start))
lengths.append(JLong.valueOf(file.length))

val partitionColumn = mutable.Map.empty[String, String]
for (i <- 0 until file.partitionValues.numFields) {
val partitionColumnValue = if (file.partitionValues.isNullAt(i)) {
ExternalCatalogUtils.DEFAULT_PARTITION_NAME
} else {
val pn = file.partitionValues.get(i, schema.fields(i).dataType)
schema.fields(i).dataType match {
case _: BinaryType =>
new String(pn.asInstanceOf[Array[Byte]], StandardCharsets.UTF_8)
case _: DateType =>
DateFormatter.apply().format(pn.asInstanceOf[Integer])
case _: TimestampType =>
TimestampFormatter
.getFractionFormatter(ZoneOffset.UTC)
.format(pn.asInstanceOf[JLong])
case _ => pn.toString
}
override def genReadSplit(
partition: InputPartition,
partitionSchemas: StructType,
fileFormat: ReadFileFormat): ReadSplit = {
partition match {
case f: FilePartition =>
val (paths, starts, lengths, partitionColumns) =
constructSplitInfo(partitionSchemas, f.files)
val preferredLocations =
SoftAffinityUtil.getFilePartitionLocations(paths.asScala.toArray, f.preferredLocations())
LocalFilesBuilder.makeLocalFiles(
f.index,
paths,
starts,
lengths,
partitionColumns,
fileFormat,
preferredLocations.toList.asJava)
}
}

private def constructSplitInfo(schema: StructType, files: Array[PartitionedFile]) = {
val paths = new JArrayList[String]()
val starts = new JArrayList[JLong]
val lengths = new JArrayList[JLong]()
val partitionColumns = new JArrayList[JMap[String, String]]
files.foreach {
file =>
paths.add(URLDecoder.decode(file.filePath.toString, StandardCharsets.UTF_8.name()))
starts.add(JLong.valueOf(file.start))
lengths.add(JLong.valueOf(file.length))

val partitionColumn = new JHashMap[String, String]()
for (i <- 0 until file.partitionValues.numFields) {
val partitionColumnValue = if (file.partitionValues.isNullAt(i)) {
ExternalCatalogUtils.DEFAULT_PARTITION_NAME
} else {
val pn = file.partitionValues.get(i, schema.fields(i).dataType)
schema.fields(i).dataType match {
case _: BinaryType =>
new String(pn.asInstanceOf[Array[Byte]], StandardCharsets.UTF_8)
case _: DateType =>
DateFormatter.apply().format(pn.asInstanceOf[Integer])
case _: TimestampType =>
TimestampFormatter
.getFractionFormatter(ZoneOffset.UTC)
.format(pn.asInstanceOf[java.lang.Long])
case _ => pn.toString
}
partitionColumn.put(schema.names(i), partitionColumnValue)
}
partitionColumns.append(partitionColumn.toMap)
}
(paths, starts, lengths, partitionColumns)
partitionColumn.put(schema.names(i), partitionColumnValue)
}
partitionColumns.add(partitionColumn)
}

val localFilesNodesWithLocations = partitions.indices.map(
i =>
partitions(i) match {
case f: FilePartition =>
val fileFormat = fileFormats(i)
val partitionSchema = partitionSchemas(i)
val (paths, starts, lengths, partitionColumns) =
constructSplitInfo(partitionSchema, f.files)
(
LocalFilesBuilder.makeLocalFiles(
f.index,
paths.asJava,
starts.asJava,
lengths.asJava,
partitionColumns.map(_.asJava).asJava,
fileFormat),
SoftAffinityUtil.getFilePartitionLocations(f))
})
wsCxt.substraitContext.initLocalFilesNodesIndex(0)
wsCxt.substraitContext.setLocalFilesNodes(localFilesNodesWithLocations.map(_._1))
val substraitPlan = wsCxt.root.toProtobuf
GlutenPartition(index, substraitPlan.toByteArray, localFilesNodesWithLocations.head._2)
(paths, starts, lengths, partitionColumns)
}

/**
Expand Down Expand Up @@ -211,8 +200,7 @@ class IteratorApiImpl extends IteratorApi with Logging {
override def genNativeFileScanRDD(
sparkContext: SparkContext,
wsCxt: WholeStageTransformContext,
fileFormat: ReadFileFormat,
inputPartitions: Seq[InputPartition],
readSplits: Seq[ReadSplit],
numOutputRows: SQLMetric,
numOutputBatches: SQLMetric,
scanTime: SQLMetric): RDD[ColumnarBatch] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,19 @@
*/
package io.glutenproject.substrait.rel;

import java.util.List;

public class ExtensionTableBuilder {
private ExtensionTableBuilder() {}

public static ExtensionTableNode makeExtensionTable(
Long minPartsNum, Long maxPartsNum, String database, String tableName, String relativePath) {
return new ExtensionTableNode(minPartsNum, maxPartsNum, database, tableName, relativePath);
Long minPartsNum,
Long maxPartsNum,
String database,
String tableName,
String relativePath,
List<String> preferredLocations) {
return new ExtensionTableNode(
minPartsNum, maxPartsNum, database, tableName, relativePath, preferredLocations);
}
}
Loading

0 comments on commit 4460e2f

Please sign in to comment.