Skip to content

Commit

Permalink
support InsertIntoHiveDirCommand in parquet write
Browse files Browse the repository at this point in the history
  • Loading branch information
JkSelf committed May 17, 2023
1 parent 52c267e commit f865602
Show file tree
Hide file tree
Showing 5 changed files with 245 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,13 @@
package org.apache.spark.sql

import io.glutenproject.execution.{ColumnarToFakeRowAdaptor, GlutenRowToArrowColumnarExec}

import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.command.DataWritingCommandExec
import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
import org.apache.spark.sql.execution.datasources.velox.VeloxParquetFileFormat
import org.apache.spark.sql.hive.execution.InsertIntoHiveDirCommand

object VeloxColumnarRules {

Expand All @@ -38,6 +38,13 @@ object VeloxColumnarRules {
} else {
plan.withNewChildren(plan.children.map(apply))
}
case command: InsertIntoHiveDirCommand =>
if (command.storage.outputFormat.get.equals(
"org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) {
rc.withNewChildren(Array(ColumnarToFakeRowAdaptor(child)))
} else {
plan.withNewChildren(plan.children.map(apply))
}
case _ => plan.withNewChildren(plan.children.map(apply))
}
case rc@DataWritingCommandExec(cmd, child) =>
Expand All @@ -60,6 +67,25 @@ object VeloxColumnarRules {
} else {
plan.withNewChildren(plan.children.map(apply))
}
case command: InsertIntoHiveDirCommand =>
if (command.storage.outputFormat.get.equals(
"org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) {
child match {
case c: AdaptiveSparkPlanExec =>
rc.withNewChildren(
Array(
AdaptiveSparkPlanExec(
ColumnarToFakeRowAdaptor(c.inputPlan),
c.context,
c.preprocessingRules,
c.isSubquery)))
case other =>
rc.withNewChildren(
Array(ColumnarToFakeRowAdaptor(GlutenRowToArrowColumnarExec(child))))
}
} else {
plan.withNewChildren(plan.children.map(apply))
}
case _ => plan.withNewChildren(plan.children.map(apply))
}
case plan: SparkPlan => plan.withNewChildren(plan.children.map(apply))
Expand Down
5 changes: 5 additions & 0 deletions package/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,11 @@
<artifactId>backends-velox</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>io.glutenproject</groupId>
<artifactId>spark-sql-columnar-shims-write</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</profile>
<profile>
Expand Down
1 change: 1 addition & 0 deletions shims/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
<activeByDefault>true</activeByDefault>
</activation>
<modules>
<module>write</module>
<module>common</module>
<module>spark32</module>
</modules>
Expand Down
76 changes: 76 additions & 0 deletions shims/write/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed 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.
-->
<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns="http://maven.apache.org/POM/4.0.0"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>io.glutenproject</groupId>
<artifactId>spark-sql-columnar-shims</artifactId>
<version>0.5.0-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

<artifactId>spark-sql-columnar-shims-write</artifactId>
<name>Gluten Shims Write</name>
<packaging>jar</packaging>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_${scala.binary.version}</artifactId>
<version>${spark32.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>io.glutenproject</groupId>
<artifactId>backends-velox</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.scalastyle</groupId>
<artifactId>scalastyle-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>com.diffplug.spotless</groupId>
<artifactId>spotless-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* 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.hive.execution

import io.glutenproject.columnarbatch.{ArrowColumnarBatches, GlutenIndicatorVector}
import io.glutenproject.execution.FakeRow
import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators
import io.glutenproject.spark.sql.execution.datasources.velox.DatasourceJniWrapper
import io.glutenproject.utils.GlutenArrowAbiUtil

import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory, VeloxWriteQueue}
import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.utils.SparkArrowUtil
import org.apache.spark.util.SerializableJobConf

import org.apache.arrow.c.ArrowSchema
import org.apache.hadoop.fs.FileStatus
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
import org.apache.parquet.hadoop.codec.CodecConfig

import java.io.IOException
import java.net.URI

/**
* `FileFormat` for writing Hive tables.
*
* TODO: implement the read logic.
*/
class HiveFileFormat(fileSinkConf: FileSinkDesc)
extends FileFormat
with DataSourceRegister
with Logging {

def this() = this(null)

override def shortName(): String = "hive"

override def inferSchema(
sparkSession: SparkSession,
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = {
throw QueryExecutionErrors.inferSchemaUnsupportedForHiveError()
}

override def prepareWrite(
sparkSession: SparkSession,
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
new OutputWriterFactory {
override def getFileExtension(context: TaskAttemptContext): String = {
CodecConfig.from(context).getCodec.getExtension + ".parquet"
}

override def newInstance(
path: String,
dataSchema: StructType,
context: TaskAttemptContext): OutputWriter = {
val originPath = path

URI.create(originPath) // validate uri
val matcher = VeloxWriteQueue.TAILING_FILENAME_REGEX.matcher(originPath)
if (!matcher.matches()) {
throw new IllegalArgumentException("illegal out put file uri: " + originPath)
}
val fileName = matcher.group(2)

val arrowSchema = SparkArrowUtil.toArrowSchema(dataSchema, SQLConf.get.sessionLocalTimeZone)
val cSchema = ArrowSchema.allocateNew(ArrowBufferAllocators.contextInstance())
var instanceId = -1L
val datasourceJniWrapper = new DatasourceJniWrapper()
val allocator = ArrowBufferAllocators.contextInstance()
try {
GlutenArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema)
instanceId =
datasourceJniWrapper.nativeInitDatasource(originPath, fileName, cSchema.memoryAddress())
} catch {
case e: IOException =>
throw new RuntimeException(e)
} finally {
cSchema.close()
}

val writeQueue =
new VeloxWriteQueue(instanceId, arrowSchema, allocator, datasourceJniWrapper, originPath)

new OutputWriter {
override def write(row: InternalRow): Unit = {
val batch = row.asInstanceOf[FakeRow].batch
if (batch.column(0).isInstanceOf[GlutenIndicatorVector]) {
val giv = batch.column(0).asInstanceOf[GlutenIndicatorVector]
giv.retain()
writeQueue.enqueue(batch)
} else {
val offloaded =
ArrowColumnarBatches.ensureOffloaded(ArrowBufferAllocators.contextInstance, batch)
writeQueue.enqueue(offloaded)
}
}

override def close(): Unit = {
writeQueue.close()
datasourceJniWrapper.close(instanceId)
}

// Do NOT add override keyword for compatibility on spark 3.1.
def path(): String = {
originPath
}
}
}
}
}
}

0 comments on commit f865602

Please sign in to comment.