Skip to content

Commit

Permalink
[GLUTEN-3547][CORE] [VL] Add native parquet writer in spark 3.4 (#3690)
Browse files Browse the repository at this point in the history
  • Loading branch information
JkSelf authored Dec 27, 2023
1 parent 1e382bd commit fd33a93
Show file tree
Hide file tree
Showing 30 changed files with 918 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType}

Expand Down Expand Up @@ -269,4 +270,8 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
SparkEnv.get.conf
.getLong(GLUTEN_MAX_SHUFFLE_READ_BYTES, GLUTEN_MAX_SHUFFLE_READ_BYTES_DEFAULT)
}

override def supportWriteFilesExec(
format: FileFormat,
fields: Array[StructField]): Option[String] = None
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
import org.apache.spark.shuffle.utils.CHShuffleUtil
import org.apache.spark.sql.{SparkSession, Strategy}
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.BuildSide
Expand All @@ -40,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec
import org.apache.spark.sql.execution.datasources.{FileFormat, WriteFilesExec}
import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.NativeWritePostRule
import org.apache.spark.sql.execution.datasources.v1.ClickHouseFileIndex
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
Expand Down Expand Up @@ -452,6 +455,16 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
CHTruncTimestampTransformer(substraitExprName, format, timestamp, timeZoneId, original)
}

override def createColumnarWriteFilesExec(
child: SparkPlan,
fileFormat: FileFormat,
partitionColumns: Seq[Attribute],
bucketSpec: Option[BucketSpec],
options: Map[String, String],
staticPartitions: TablePartitionSpec): WriteFilesExec = {
throw new UnsupportedOperationException("ColumnarWriteFilesExec is not support in ch backend.")
}

/**
* Define whether the join operator is fallback because of the join operator is not supported by
* backend
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import io.glutenproject.execution._
import io.glutenproject.expression._
import io.glutenproject.expression.ConverterUtils.FunctionConfig
import io.glutenproject.memory.nmm.NativeMemoryManagers
import io.glutenproject.sql.shims.SparkShimLoader
import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode, IfThenNode}
import io.glutenproject.vectorized.{ColumnarBatchSerializer, ColumnarBatchSerializerJniWrapper}

Expand All @@ -34,14 +35,17 @@ import org.apache.spark.shuffle.utils.ShuffleUtil
import org.apache.spark.sql.{SparkSession, Strategy}
import org.apache.spark.sql.catalyst.{AggregateFunctionRewriteRule, FunctionIdentifier}
import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, CreateNamedStruct, ElementAt, Expression, ExpressionInfo, GetArrayItem, GetMapValue, GetStructField, Literal, NamedExpression, StringSplit, StringTrim}
import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, HLLAdapter}
import org.apache.spark.sql.catalyst.optimizer.BuildSide
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.{ColumnarBuildSideRelation, SparkPlan}
import org.apache.spark.sql.execution.{ColumnarBuildSideRelation, SparkPlan, VeloxColumnarWriteFilesExec}
import org.apache.spark.sql.execution.datasources.{FileFormat, WriteFilesExec}
import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.NativeWritePostRule
import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec
import org.apache.spark.sql.execution.joins.BuildSideRelation
Expand Down Expand Up @@ -245,6 +249,22 @@ class SparkPlanExecApiImpl extends SparkPlanExecApi {
ShuffleUtil.genColumnarShuffleWriter(parameters)
}

override def createColumnarWriteFilesExec(
child: SparkPlan,
fileFormat: FileFormat,
partitionColumns: Seq[Attribute],
bucketSpec: Option[BucketSpec],
options: Map[String, String],
staticPartitions: TablePartitionSpec): WriteFilesExec = {
new VeloxColumnarWriteFilesExec(
child,
fileFormat,
partitionColumns,
bucketSpec,
options,
staticPartitions)
}

/**
* Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec.
*
Expand Down Expand Up @@ -477,7 +497,12 @@ class SparkPlanExecApiImpl extends SparkPlanExecApi {
* @return
*/
override def genExtendedColumnarPostRules(): List[SparkSession => Rule[SparkPlan]] = {
List(spark => NativeWritePostRule(spark))
val Array(major, minor, _) = SparkShimLoader.getSparkShims.getShimDescriptor.toString.split('.')
if (major.toInt > 3 || (major.toInt == 3 && (minor.toInt >= 4))) {
List()
} else {
List(spark => NativeWritePostRule(spark))
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,8 @@ import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.execution.{ProjectExec, SparkPlan}
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand
import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand}
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.expression.UDFResolver
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -114,6 +115,58 @@ object BackendSettings extends BackendSettingsApi {
}
}

override def supportWriteFilesExec(
format: FileFormat,
fields: Array[StructField]): Option[String] = {
def validateCompressionCodec(): Option[String] = {
// Velox doesn't support brotli and lzo.
val unSupportedCompressions = Set("brotli, lzo")
if (unSupportedCompressions.contains(SQLConf.get.parquetCompressionCodec.toLowerCase())) {
Some("brotli or lzo compression codec is not support in velox backend.")
} else {
None
}
}

// Validate if all types are supported.
def validateDateTypes(fields: Array[StructField]): Option[String] = {
fields.flatMap {
field =>
field.dataType match {
case _: TimestampType => Some("TimestampType")
case struct: StructType if validateDateTypes(struct.fields).nonEmpty =>
Some("StructType(TimestampType)")
case array: ArrayType if array.elementType.isInstanceOf[TimestampType] =>
Some("ArrayType(TimestampType)")
case map: MapType
if map.keyType.isInstanceOf[TimestampType] ||
map.valueType.isInstanceOf[TimestampType] =>
Some("MapType(TimestampType)")
case _ => None
}
}.headOption
}

def validateFieldMetadata(fields: Array[StructField]): Option[String] = {
if (fields.exists(!_.metadata.equals(Metadata.empty))) {
Some("StructField contain the metadata information.")
} else None
}

def validateFileFormat(format: FileFormat): Option[String] = {
format match {
case _: ParquetFileFormat => None
case _: FileFormat => Some("Only parquet fileformat is supported in native write.")
}
}

val fileFormat = validateFileFormat(format)
val metadata = validateFieldMetadata(fields)
val dataTypes = validateDateTypes(fields)
val compression = validateCompressionCodec()
compression.orElse(dataTypes).orElse(metadata).orElse(fileFormat)
}

override def supportExpandExec(): Boolean = true

override def supportSortExec(): Boolean = true
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,146 @@
/*
* 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

import io.glutenproject.columnarbatch.ColumnarBatches
import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators

import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.write.WriterCommitMessage
import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, FileFormat, PartitioningUtils, WriteFilesExec, WriteFilesSpec, WriteTaskResult}
import org.apache.spark.sql.vectorized.ColumnarBatch

import shaded.parquet.com.fasterxml.jackson.databind.ObjectMapper

import scala.collection.mutable

class VeloxColumnarWriteFilesExec(
child: SparkPlan,
fileFormat: FileFormat,
partitionColumns: Seq[Attribute],
bucketSpec: Option[BucketSpec],
options: Map[String, String],
staticPartitions: TablePartitionSpec)
extends WriteFilesExec(
child,
fileFormat,
partitionColumns,
bucketSpec,
options,
staticPartitions) {

override def supportsColumnar(): Boolean = true

override def doExecuteWrite(writeFilesSpec: WriteFilesSpec): RDD[WriterCommitMessage] = {
assert(child.supportsColumnar)

// We need to pass the WritePath to the Velox TableWriter in the doTransform
// method of the WriteTransformer. However, the WritePath is not accessible
// during the planning phase in the WriteTransformer, and can only be obtained
// during the actual execution, specifically in the doExecuteWrite method of
// ColumnarWriteFilesExec, where it is available within the WriteFilesSpec.
// Therefore, we use this hack method to pass the writePath.
child.session.sparkContext.setLocalProperty("writePath", writeFilesSpec.description.path)
child.executeColumnar().mapPartitionsInternal {
iter =>
// Currently, the cb contains three columns: row, fragments, and context.
// The first row in the row column contains the number of written numRows.
// The fragments column contains detailed information about the file writes.
// The json can be as following:
// {
// "inMemoryDataSizeInBytes":0,
// "containsNumberedFileNames":true,
// "onDiskDataSizeInBytes":307,
// "fileWriteInfos":[
// {
// "fileSize":307,
// "writeFileName":
// "Gluten_Stage_1_TID_2_0_2_d1db3b31-4f99-41cb-a4e7-3b8607506168.parquet",
// "targetFileName":
// "Gluten_Stage_1_TID_2_0_2_d1db3b31-4f99-41cb-a4e7-3b8607506168.parquet"
// }
// ],
// "writePath":"file:/home/gluten/spark-warehouse/inserttable/part1=1/part2=1",
// "rowCount":1,
// "targetPath":"file:/home/gluten/spark-warehouse/inserttable/part1=1/part2=1",
// "updateMode":"NEW",
// "name":"part1=1/part2=1"
// }
assert(iter.hasNext)
val cb = iter.next()
val loadedCb = ColumnarBatches.ensureLoaded(ArrowBufferAllocators.contextInstance, cb)
val numWrittenRows = loadedCb.column(0).getLong(0)

var updatedPartitions = Set.empty[String]
val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]()
var numBytes = 0L
for (i <- 0 until loadedCb.numRows() - 1) {
val fragments = loadedCb.column(1).getUTF8String(i + 1)
val objectMapper = new ObjectMapper()
val jsonObject = objectMapper.readTree(fragments.toString)

val fileWriteInfos = jsonObject.get("fileWriteInfos").elements()
if (jsonObject.get("fileWriteInfos").elements().hasNext) {
val writeInfo = fileWriteInfos.next();
numBytes += writeInfo.get("fileSize").longValue()
// Get partition information.
if (jsonObject.get("name").textValue().nonEmpty) {
val targetFileName = writeInfo.get("targetFileName").textValue()
val partitionDir = jsonObject.get("name").textValue()
updatedPartitions += partitionDir
val tmpOutputPath =
writeFilesSpec.description.path + "/" + partitionDir + "/" + targetFileName
val absOutputPathObject =
writeFilesSpec.description.customPartitionLocations.get(
PartitioningUtils.parsePathFragment(partitionDir))
if (absOutputPathObject.nonEmpty) {
val absOutputPath = absOutputPathObject.get + "/" + targetFileName
addedAbsPathFiles(tmpOutputPath) = absOutputPath
}
}
}
}

// TODO: need to get the partition Internal row?
val stats = BasicWriteTaskStats(Seq.empty, loadedCb.numRows() - 1, numBytes, numWrittenRows)
val summary =
ExecutedWriteSummary(updatedPartitions = updatedPartitions, stats = Seq(stats))

val result = WriteTaskResult(
new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions),
summary)
Iterator.single(result)
}
}

override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().")
}

override protected def withNewChildInternal(newChild: SparkPlan): WriteFilesExec =
new VeloxColumnarWriteFilesExec(
newChild,
fileFormat,
partitionColumns,
bucketSpec,
options,
staticPartitions)
}
Original file line number Diff line number Diff line change
Expand Up @@ -445,16 +445,20 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit
}
}

ignore("Velox Parquet Write") {
test("Velox Parquet Write") {
withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) {
withTempDir {
dir =>
val write_path = dir.toURI.getPath
val data_path = getClass.getResource("/").getPath + "/data-type-validation-data/type1"
val df = spark.read.format("parquet").load(data_path)
// Velox native write doesn't support Timestamp type.
val df = spark.read.format("parquet").load(data_path).drop("timestamp")
df.write.mode("append").format("parquet").save(write_path)
val parquetDf = spark.read
.format("parquet")
.load(write_path)
checkAnswer(parquetDf, df)
}
}

}
}
Loading

0 comments on commit fd33a93

Please sign in to comment.