();
RowIterator resIter = transKernel.createKernelWithRowIterator(substraitPlan.toByteArray(),
inBatchIters);
while (resIter.hasNext()) {
diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala
new file mode 100644
index 000000000000..e807ff6305f4
--- /dev/null
+++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala
@@ -0,0 +1,645 @@
+/*
+ * 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 io.glutenproject.benchmarks
+
+import java.io.File
+
+import scala.collection.mutable.ArrayBuffer
+import scala.io.Source
+
+import io.glutenproject.GlutenConfig
+
+import org.apache.spark.sql.SparkSession
+
+object DSV2BenchmarkTest {
+
+ val tableName = "lineitem_ch"
+
+ def main(args: Array[String]): Unit = {
+
+ val (parquetFilesPath, fileFormat,
+ executedCnt, configed, sqlFilePath, stopFlagFile,
+ createTable, metaRootPath) = if (args.length > 0) {
+ (args(0), args(1), args(2).toInt, true, args(3), args(4), args(5).toBoolean, args(6))
+ } else {
+ val rootPath = this.getClass.getResource("/").getPath
+ val resourcePath = rootPath + "../../../../jvm/src/test/resources/"
+ val dataPath = resourcePath + "/tpch-data/"
+ val queryPath = resourcePath + "/queries/"
+ //(new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q06.sql", "", true,
+ //"/data1/gazelle-jni-warehouse")
+ ("/data1/test_output/tpch-data-sf10", "parquet", 1, false, queryPath + "q01.sql", "",
+ true, "/data1/gazelle-jni-warehouse")
+ }
+
+ val (warehouse, metaStorePathAbsolute, hiveMetaStoreDB) = if (!metaRootPath.isEmpty) {
+ (metaRootPath + "/spark-warehouse", metaRootPath + "/meta",
+ metaRootPath + "/meta/metastore_db")
+ } else {
+ ("/tmp/spark-warehouse", "/tmp/meta", "/tmp/meta/metastore_db")
+ }
+
+ if (!warehouse.isEmpty) {
+ val warehouseDir = new File(warehouse)
+ if (!warehouseDir.exists()) {
+ warehouseDir.mkdirs()
+ }
+ val hiveMetaStoreDBDir = new File(metaStorePathAbsolute)
+ if (!hiveMetaStoreDBDir.exists()) {
+ hiveMetaStoreDBDir.mkdirs()
+ }
+ }
+
+ val sqlStr = Source.fromFile(new File(sqlFilePath), "UTF-8")
+
+ val sessionBuilderTmp = SparkSession
+ .builder()
+ .appName("Gluten-Benchmark")
+
+ val sessionBuilder = if (!configed) {
+ val sessionBuilderTmp1 = sessionBuilderTmp
+ .master("local[3]")
+ .config("spark.driver.memory", "4G")
+ .config("spark.driver.memoryOverhead", "6G")
+ .config("spark.serializer", "org.apache.spark.serializer.JavaSerializer")
+ .config("spark.default.parallelism", 1)
+ .config("spark.sql.shuffle.partitions", 1)
+ .config("spark.sql.adaptive.enabled", "false")
+ .config("spark.sql.files.maxPartitionBytes", 1024 << 10 << 10) // default is 128M
+ .config("spark.sql.files.minPartitionNum", "1")
+ .config("spark.sql.parquet.filterPushdown", "true")
+ .config("spark.locality.wait", "0s")
+ .config("spark.sql.sources.ignoreDataLocality", "true")
+ .config("spark.sql.parquet.enableVectorizedReader", "true")
+ //.config("spark.sql.sources.useV1SourceList", "avro")
+ .config("spark.memory.fraction", "0.3")
+ .config("spark.memory.storageFraction", "0.3")
+ //.config("spark.sql.parquet.columnarReaderBatchSize", "20000")
+ .config("spark.plugins", "io.glutenproject.GlutenPlugin")
+ .config("spark.sql.catalog.spark_catalog",
+ "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog")
+ .config("spark.databricks.delta.maxSnapshotLineageLength", 20)
+ .config("spark.databricks.delta.snapshotPartitions", 1)
+ .config("spark.databricks.delta.properties.defaults.checkpointInterval", 5)
+ .config("spark.databricks.delta.stalenessLimit", 3600 * 1000)
+ .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
+ //.config("spark.sql.execution.arrow.maxRecordsPerBatch", "20000")
+ .config("spark.gluten.sql.columnar.columnartorow", "false")
+ .config("spark.gluten.sql.columnar.backend.lib", "ch")
+ .config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true")
+ .config(GlutenConfig.GLUTEN_LOAD_ARROW, "false")
+ .config(GlutenConfig.GLUTEN_LIB_PATH,
+ "/usr/local/clickhouse/lib/libch.so")
+ .config("spark.gluten.sql.columnar.iterator", "true")
+ .config("spark.gluten.sql.columnar.hashagg.enablefinal", "true")
+ .config("spark.gluten.sql.enable.native.validation", "false")
+ //.config("spark.sql.planChangeLog.level", "info")
+ .config("spark.sql.columnVector.offheap.enabled", "true")
+ .config("spark.memory.offHeap.enabled", "true")
+ .config("spark.memory.offHeap.size", "6442450944")
+ .config("spark.io.compression.codec", "LZ4")
+
+ if (!warehouse.isEmpty) {
+ sessionBuilderTmp1.config("spark.sql.warehouse.dir", warehouse)
+ .config("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$hiveMetaStoreDB;create=true")
+ .enableHiveSupport()
+ } else {
+ sessionBuilderTmp1.enableHiveSupport()
+ }
+ } else {
+ sessionBuilderTmp
+ }
+
+ val spark = sessionBuilder.getOrCreate()
+ if (!configed) {
+ spark.sparkContext.setLogLevel("WARN")
+ }
+
+ val createTbl = false
+ if (createTbl) {
+ createClickHouseTables(spark, parquetFilesPath, fileFormat)
+ createLocationClickHouseTable(spark)
+ // createTables(spark, parquetFilesPath, fileFormat)
+ }
+ val refreshTable = false
+ if (refreshTable) {
+ refreshClickHouseTable(spark)
+ }
+ selectClickHouseTable(spark, executedCnt, sqlStr.mkString)
+ //selectLocationClickHouseTable(spark, executedCnt, sqlStr.mkString)
+ //testSerializeFromObjectExec(spark)
+
+ selectQ1ClickHouseTable(spark, executedCnt, sqlStr.mkString)
+ //selectQ1LocationClickHouseTable(spark, executedCnt, sqlStr.mkString)
+
+ //testSparkTPCH(spark)
+ //testSQL(spark, parquetFilesPath, fileFormat, executedCnt, sqlStr.mkString)
+
+ System.out.println("waiting for finishing")
+ if (stopFlagFile.isEmpty) {
+ Thread.sleep(1800000)
+ } else {
+ while ((new File(stopFlagFile)).exists()) {
+ Thread.sleep(1000)
+ }
+ }
+ spark.stop()
+ System.out.println("finished")
+ }
+
+ def testSerializeFromObjectExec(spark: SparkSession): Unit = {
+ //spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ val tookTimeArr = Array(12, 23, 56, 100, 500, 20)
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def createClickHouseTables(spark: SparkSession,
+ parquetFilesPath: String, fileFormat: String): Unit = {
+ spark.sql(
+ """
+ | show databases
+ |""".stripMargin).show(100, false)
+
+ spark.sql(
+ """
+ | show tables
+ |""".stripMargin).show(100, false)
+
+ spark.sql(
+ s"""
+ | USE default
+ |""".stripMargin).show(100, false)
+
+ // Clear up old session
+ spark.sql(s"DROP TABLE IF EXISTS $tableName")
+
+ // Create a table
+ println("Creating a table")
+ // PARTITIONED BY (age)
+ // engine='MergeTree' or engine='Parquet'
+ spark.sql(
+ s"""
+ | CREATE TABLE IF NOT EXISTS $tableName (
+ | l_orderkey bigint,
+ | l_partkey bigint,
+ | l_suppkey bigint,
+ | l_linenumber bigint,
+ | l_quantity double,
+ | l_extendedprice double,
+ | l_discount double,
+ | l_tax double,
+ | l_returnflag string,
+ | l_linestatus string,
+ | l_shipdate date,
+ | l_commitdate date,
+ | l_receiptdate date,
+ | l_shipinstruct string,
+ | l_shipmode string,
+ | l_comment string)
+ | USING clickhouse
+ | TBLPROPERTIES (engine='MergeTree'
+ | )
+ |""".stripMargin)
+
+ spark.sql(
+ """
+ | show tables
+ |""".stripMargin).show(100, false)
+
+ spark.sql(
+ s"""
+ | desc formatted ${tableName}
+ |""".stripMargin).show(100, false)
+
+ }
+
+ def createLocationClickHouseTable(spark: SparkSession): Unit = {
+ spark.sql(
+ s"""
+ | USE default
+ |""".stripMargin).show(100, false)
+
+ spark.sql(
+ """
+ | show tables
+ |""".stripMargin).show(100, false)
+
+ // Clear up old session
+ spark.sql(s"DROP TABLE IF EXISTS ch_clickhouse")
+
+ // Create a table
+ // PARTITIONED BY (age)
+ // engine='MergeTree' or engine='Parquet'
+ spark.sql(
+ s"""
+ | CREATE TABLE IF NOT EXISTS ch_clickhouse (
+ | l_orderkey bigint,
+ | l_partkey bigint,
+ | l_suppkey bigint,
+ | l_linenumber bigint,
+ | l_quantity double,
+ | l_extendedprice double,
+ | l_discount double,
+ | l_tax double,
+ | l_returnflag string,
+ | l_linestatus string,
+ | l_shipdate date,
+ | l_commitdate date,
+ | l_receiptdate date,
+ | l_shipinstruct string,
+ | l_shipmode string,
+ | l_comment string)
+ | USING clickhouse
+ | TBLPROPERTIES (engine='MergeTree'
+ | )
+ | LOCATION '/data1/gazelle-jni-warehouse/ch_clickhouse'
+ |""".stripMargin)
+
+ spark.sql(
+ """
+ | show tables
+ |""".stripMargin).show(100, false)
+ }
+
+ def refreshClickHouseTable(spark: SparkSession): Unit = {
+ spark.sql(
+ s"""
+ | refresh table ${tableName}
+ |""".stripMargin).show(100, false)
+ spark.sql(
+ s"""
+ | desc formatted ${tableName}
+ |""".stripMargin).show(100, false)
+ spark.sql(
+ s"""
+ | refresh table ch_clickhouse
+ |""".stripMargin).show(100, false)
+ spark.sql(
+ s"""
+ | desc formatted ch_clickhouse
+ |""".stripMargin).show(100, false)
+ }
+
+ def selectClickHouseTable(spark: SparkSession, executedCnt: Int,
+ sql: String): Unit = {
+ val tookTimeArr = ArrayBuffer[Long]()
+ for (i <- 1 to executedCnt) {
+ val startTime = System.nanoTime()
+ spark.sql(
+ s"""
+ |SELECT
+ | sum(l_extendedprice * l_discount) AS revenue
+ |FROM
+ | ${tableName}
+ |WHERE
+ | l_shipdate >= date'1994-01-01'
+ | AND l_shipdate < date'1994-01-01' + interval 1 year
+ | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
+ | AND l_quantity < 24;
+ |""".stripMargin).show(200, false)
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+
+ println(tookTimeArr.mkString(","))
+
+ // spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def selectLocationClickHouseTable(spark: SparkSession, executedCnt: Int,
+ sql: String): Unit = {
+ val tookTimeArr = ArrayBuffer[Long]()
+ for (i <- 1 to executedCnt) {
+ val startTime = System.nanoTime()
+ spark.sql(
+ s"""
+ |SELECT
+ | sum(l_extendedprice * l_discount) AS revenue
+ |FROM
+ | ch_clickhouse
+ |WHERE
+ | l_shipdate >= date'1994-01-01'
+ | AND l_shipdate < date'1994-01-01' + interval 1 year
+ | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
+ | AND l_quantity < 24;
+ |""".stripMargin).show(200, false)
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+
+ println(tookTimeArr.mkString(","))
+
+ //spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def selectQ1ClickHouseTable(spark: SparkSession, executedCnt: Int,
+ sql: String): Unit = {
+ val tookTimeArr = ArrayBuffer[Long]()
+ for (i <- 1 to executedCnt) {
+ val startTime = System.nanoTime()
+ spark.sql(
+ s"""
+ |SELECT
+ | l_returnflag,
+ | l_linestatus,
+ | sum(l_quantity) AS sum_qty,
+ | sum(l_extendedprice) AS sum_base_price,
+ | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+ | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+ | avg(l_quantity) AS avg_qty,
+ | avg(l_extendedprice) AS avg_price,
+ | avg(l_discount) AS avg_disc,
+ | count(*) AS count_order
+ |FROM
+ | ${tableName}
+ |WHERE
+ | l_shipdate <= date'1998-09-02' - interval 1 day
+ |GROUP BY
+ | l_returnflag,
+ | l_linestatus
+ |ORDER BY
+ | l_returnflag,
+ | l_linestatus;
+ |""".stripMargin).show(200, false)
+ // can not use .collect(), will lead to error.
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+
+ println(tookTimeArr.mkString(","))
+
+ // spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def selectQ1LocationClickHouseTable(spark: SparkSession, executedCnt: Int,
+ sql: String): Unit = {
+ val tookTimeArr = ArrayBuffer[Long]()
+ for (i <- 1 to executedCnt) {
+ val startTime = System.nanoTime()
+ spark.sql(
+ s"""
+ |SELECT
+ | l_returnflag,
+ | l_linestatus,
+ | sum(l_quantity) AS sum_qty,
+ | sum(l_extendedprice) AS sum_base_price,
+ | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+ | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+ | avg(l_quantity) AS avg_qty,
+ | avg(l_extendedprice) AS avg_price,
+ | avg(l_discount) AS avg_disc,
+ | count(*) AS count_order
+ |FROM
+ | ch_clickhouse
+ |WHERE
+ | l_shipdate <= date'1998-09-02' - interval 1 day
+ |GROUP BY
+ | l_returnflag,
+ | l_linestatus
+ |ORDER BY
+ | l_returnflag,
+ | l_linestatus;
+ |""".stripMargin).show(200, false)
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+
+ println(tookTimeArr.mkString(","))
+
+ //spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def testSQL(spark: SparkSession, parquetFilesPath: String,
+ fileFormat: String, executedCnt: Int,
+ sql: String): Unit = {
+ /*spark.sql(
+ s"""
+ | show tables
+ |""".stripMargin).show(100, false)*/
+
+ val tookTimeArr = ArrayBuffer[Long]()
+ for (i <- 1 to executedCnt) {
+ val startTime = System.nanoTime()
+ spark.sql(sql).show(200, false)
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+
+ println(tookTimeArr.mkString(","))
+
+ //spark.conf.set("spark.gluten.sql.enable.native.engine", "false")
+ import spark.implicits._
+ val df = spark.sparkContext.parallelize(tookTimeArr.toSeq, 1).toDF("time")
+ df.summary().show(100, false)
+ }
+
+ def testSparkTPCH(spark: SparkSession): Unit = {
+ val tookTimeArr = ArrayBuffer[Long]()
+ val rootPath = this.getClass.getResource("/").getPath
+ val resourcePath = rootPath + "../../../src/test/resources/"
+ val queryPath = resourcePath + "/queries/"
+ for (i <- 1 to 22) {
+ val startTime = System.nanoTime()
+ val sqlFile = queryPath + "q" + "%02d".format(i) + ".sql"
+ println(sqlFile)
+ val sqlStr = Source.fromFile(new File(sqlFile), "UTF-8")
+ spark.sql(sqlStr.mkString).collect()
+ val tookTime = (System.nanoTime() - startTime) / 1000000
+ println(s"Execute ${i} time, time: ${tookTime}")
+ tookTimeArr += tookTime
+ }
+ println(tookTimeArr.mkString(","))
+ }
+
+ def createTables(spark: SparkSession, parquetFilesPath: String, fileFormat: String): Unit = {
+ val dataSourceMap = Map(
+ "customer" -> spark.read.format(fileFormat).load(parquetFilesPath + "/customer"),
+
+ "lineitem" -> spark.read.format(fileFormat).load(parquetFilesPath + "/lineitem"),
+
+ "nation" -> spark.read.format(fileFormat).load(parquetFilesPath + "/nation"),
+
+ "region" -> spark.read.format(fileFormat).load(parquetFilesPath + "/region"),
+
+ "orders" -> spark.read.format(fileFormat).load(parquetFilesPath + "/order"),
+
+ "part" -> spark.read.format(fileFormat).load(parquetFilesPath + "/part"),
+
+ "partsupp" -> spark.read.format(fileFormat).load(parquetFilesPath + "/partsupp"),
+
+ "supplier" -> spark.read.format(fileFormat).load(parquetFilesPath + "/supplier"))
+
+ val customerData = parquetFilesPath + "/customer"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS customer (
+ | c_custkey bigint,
+ | c_name string,
+ | c_address string,
+ | c_nationkey bigint,
+ | c_phone string,
+ | c_acctbal double,
+ | c_mktsegment string,
+ | c_comment string)
+ | STORED AS PARQUET LOCATION '${customerData}'
+ |""".stripMargin).show(1, false)
+
+ val lineitemData = parquetFilesPath + "/lineitem"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem (
+ | l_orderkey bigint,
+ | l_partkey bigint,
+ | l_suppkey bigint,
+ | l_linenumber bigint,
+ | l_quantity double,
+ | l_extendedprice double,
+ | l_discount double,
+ | l_tax double,
+ | l_returnflag string,
+ | l_linestatus string,
+ | l_shipdate date,
+ | l_commitdate date,
+ | l_receiptdate date,
+ | l_shipinstruct string,
+ | l_shipmode string,
+ | l_comment string)
+ | STORED AS PARQUET LOCATION '${lineitemData}'
+ |""".stripMargin).show(1, false)
+
+ val nationData = parquetFilesPath + "/nation"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS nation (
+ | n_nationkey bigint,
+ | n_name string,
+ | n_regionkey bigint,
+ | n_comment string)
+ | STORED AS PARQUET LOCATION '${nationData}'
+ |""".stripMargin).show(1, false)
+
+ val regionData = parquetFilesPath + "/region"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS region (
+ | r_regionkey bigint,
+ | r_name string,
+ | r_comment string)
+ | STORED AS PARQUET LOCATION '${regionData}'
+ |""".stripMargin).show(1, false)
+
+ val ordersData = parquetFilesPath + "/order"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS orders (
+ | o_orderkey bigint,
+ | o_custkey bigint,
+ | o_orderstatus string,
+ | o_totalprice double,
+ | o_orderdate date,
+ | o_orderpriority string,
+ | o_clerk string,
+ | o_shippriority bigint,
+ | o_comment string)
+ | STORED AS PARQUET LOCATION '${ordersData}'
+ |""".stripMargin).show(1, false)
+
+ val partData = parquetFilesPath + "/part"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS part (
+ | p_partkey bigint,
+ | p_name string,
+ | p_mfgr string,
+ | p_brand string,
+ | p_type string,
+ | p_size bigint,
+ | p_container string,
+ | p_retailprice double,
+ | p_comment string)
+ | STORED AS PARQUET LOCATION '${partData}'
+ |""".stripMargin).show(1, false)
+
+ val partsuppData = parquetFilesPath + "/partsupp"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp (
+ | ps_partkey bigint,
+ | ps_suppkey bigint,
+ | ps_availqty bigint,
+ | ps_supplycost double,
+ | ps_comment string)
+ | STORED AS PARQUET LOCATION '${partsuppData}'
+ |""".stripMargin).show(1, false)
+
+ val supplierData = parquetFilesPath + "/supplier"
+ spark.sql(
+ s"""
+ | CREATE EXTERNAL TABLE IF NOT EXISTS supplier (
+ | s_suppkey bigint,
+ | s_name string,
+ | s_address string,
+ | s_nationkey bigint,
+ | s_phone string,
+ | s_acctbal double,
+ | s_comment string)
+ | STORED AS PARQUET LOCATION '${supplierData}'
+ |""".stripMargin).show(1, false)
+
+ /*spark.sql(
+ s"""
+ | show databases;
+ |""".stripMargin).show(100, false)
+ spark.sql(
+ s"""
+ | show tables;
+ |""".stripMargin).show(100, false)
+ dataSourceMap.foreach {
+ case (key, value) => {
+ println(s"----------------create table $key")
+ spark.sql(
+ s"""
+ | desc $key;
+ |""".stripMargin).show(100, false)
+ spark.sql(
+ s"""
+ | select count(1) from $key;
+ |""".stripMargin).show(10, false)
+ }
+ }*/
+ }
+}
diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala
similarity index 99%
rename from jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala
rename to backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala
index 7d5e1bf78c5e..0a28cfef6761 100644
--- a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala
+++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2ColumnarBenchmarkTest.scala
@@ -17,13 +17,15 @@
package io.glutenproject.benchmarks
-import io.glutenproject.GlutenConfig
-import org.apache.spark.sql.SparkSession
-
import java.io.File
+
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
+import io.glutenproject.GlutenConfig
+
+import org.apache.spark.sql.SparkSession
+
object DSV2ColumnarBenchmarkTest {
val tableName = "lineitem_ch"
@@ -99,7 +101,7 @@ object DSV2ColumnarBenchmarkTest {
.config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true")
.config(GlutenConfig.GLUTEN_LOAD_ARROW, "false")
.config(GlutenConfig.GLUTEN_LIB_PATH,
- "/home/saber/Documents/github/ClickHouse/cmake-build-relwithdebinfo/utils/local-engine/libch.so")
+ "/usr/local/clickhouse/lib/libch.so")
.config("spark.gluten.sql.columnar.iterator", "true")
//.config("spark.sql.planChangeLog.level", "info")
.config("spark.sql.columnVector.offheap.enabled", "true")
diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala
similarity index 82%
rename from jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala
rename to backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala
index 6d65ecac6b7b..a3af9e950189 100644
--- a/jvm/src/test/scala/io/glutenproject/benchmarks/DSV2BenchmarkTest.scala
+++ b/backends-clickhouse/src/test/scala/io/glutenproject/benchmarks/DSV2Q1ColumnarBenchmarkTest.scala
@@ -26,7 +26,7 @@ import io.glutenproject.GlutenConfig
import org.apache.spark.sql.SparkSession
-object DSV2BenchmarkTest {
+object DSV2Q1ColumnarBenchmarkTest {
val tableName = "lineitem_ch"
@@ -41,8 +41,8 @@ object DSV2BenchmarkTest {
val resourcePath = rootPath + "../../../src/test/resources/"
val dataPath = resourcePath + "/tpch-data/"
val queryPath = resourcePath + "/queries/"
- (new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q06.sql", "", true,
- "/data1/gluten-warehouse")
+ (new File(dataPath).getAbsolutePath, "parquet", 1, false, queryPath + "q01.sql", "", true,
+ "/tmp/gluten-warehouse")
}
val (warehouse, metaStorePathAbsolute, hiveMetaStoreDB) = if (!metaRootPath.isEmpty) {
@@ -71,12 +71,12 @@ object DSV2BenchmarkTest {
val sessionBuilder = if (!configed) {
val sessionBuilderTmp1 = sessionBuilderTmp
- .master("local[3]")
+ .master("local[1]")
.config("spark.driver.memory", "4G")
.config("spark.driver.memoryOverhead", "6G")
.config("spark.serializer", "org.apache.spark.serializer.JavaSerializer")
.config("spark.default.parallelism", 1)
- .config("spark.sql.shuffle.partitions", 1)
+ .config("spark.sql.shuffle.partitions",1)
.config("spark.sql.adaptive.enabled", "false")
.config("spark.sql.files.maxPartitionBytes", 1024 << 10 << 10) // default is 128M
.config("spark.sql.files.minPartitionNum", "1")
@@ -95,18 +95,20 @@ object DSV2BenchmarkTest {
.config("spark.databricks.delta.snapshotPartitions", 1)
.config("spark.databricks.delta.properties.defaults.checkpointInterval", 5)
.config("spark.databricks.delta.stalenessLimit", 3600 * 1000)
- //.config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
+ .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
//.config("spark.sql.execution.arrow.maxRecordsPerBatch", "20000")
.config("spark.gluten.sql.columnar.columnartorow", "false")
.config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true")
.config(GlutenConfig.GLUTEN_LOAD_ARROW, "false")
.config(GlutenConfig.GLUTEN_LIB_PATH,
- "path_to_clickhouse_engine/libch.so")
- .config("spark.gluten.sql.columnar.iterator", "false")
+ "/usr/local/clickhouse/lib/libch.so")
+ .config("spark.gluten.sql.columnar.iterator", "true")
//.config("spark.sql.planChangeLog.level", "info")
.config("spark.sql.columnVector.offheap.enabled", "true")
.config("spark.memory.offHeap.enabled", "true")
.config("spark.memory.offHeap.size", "6442450944")
+ .config("spark.io.compression.codec", "LZ4")
+ .config("spark.gluten.sql.columnar.backend.lib", "clickhouse")
if (!warehouse.isEmpty) {
sessionBuilderTmp1.config("spark.sql.warehouse.dir", warehouse)
@@ -126,14 +128,14 @@ object DSV2BenchmarkTest {
val createTbl = true
if (createTbl) {
- createClickHouseTables(spark, parquetFilesPath, fileFormat)
+// createClickHouseTables(spark, parquetFilesPath, fileFormat)
createLocationClickHouseTable(spark)
}
val refreshTable = true
if (refreshTable) {
refreshClickHouseTable(spark)
}
- selectClickHouseTable(spark, executedCnt, sqlStr.mkString)
+// selectClickHouseTable(spark, executedCnt, sqlStr.mkString)
selectLocationClickHouseTable(spark, executedCnt, sqlStr.mkString)
System.out.println("waiting for finishing")
@@ -247,7 +249,7 @@ object DSV2BenchmarkTest {
| USING clickhouse
| TBLPROPERTIES (engine='MergeTree'
| )
- | LOCATION '/data1/gluten-warehouse/ch_clickhouse'
+ | LOCATION '/home/saber/Documents/data/mergetree'
|""".stripMargin)
spark.sql(
@@ -282,15 +284,27 @@ object DSV2BenchmarkTest {
val startTime = System.nanoTime()
spark.sql(
s"""
- |SELECT
- | sum(l_extendedprice * l_discount) AS revenue
- |FROM
- | ${tableName}
- |WHERE
- | l_shipdate >= date'1994-01-01'
- | AND l_shipdate < date'1994-01-01' + interval 1 year
- | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
- | AND l_quantity < 24;
+ |SELECT
+ | l_returnflag,
+ | l_linestatus,
+ | sum(l_quantity) as sum_qty,
+ | sum(l_extendedprice) as sum_base_price,
+ | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
+ | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
+ | avg(l_quantity) as avg_qty,
+ | avg(l_extendedprice) as avg_price,
+ | avg(l_discount) as avg_disc,
+ | count(*) as count_order
+ |FROM
+ | ${tableName}
+ |WHERE
+ | l_shipdate <= date '1998-12-01' - interval '90' day
+ |GROUP BY
+ | l_returnflag,
+ | l_linestatus
+ |ORDER BY
+ | l_returnflag,
+ | l_linestatus;
|""".stripMargin).show(200, false)
val tookTime = (System.nanoTime() - startTime) / 1000000
println(s"Execute ${i} time, time: ${tookTime}")
@@ -313,14 +327,26 @@ object DSV2BenchmarkTest {
spark.sql(
s"""
|SELECT
- | sum(l_extendedprice * l_discount) AS revenue
+ | l_returnflag,
+ | l_linestatus,
+ | sum(l_quantity) as sum_qty,
+ | sum(l_extendedprice) as sum_base_price,
+ | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price,
+ | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge,
+ | avg(l_quantity) as avg_qty,
+ | avg(l_extendedprice) as avg_price,
+ | avg(l_discount) as avg_disc,
+ | count(*) as count_order
|FROM
| ch_clickhouse
|WHERE
- | l_shipdate >= date'1994-01-01'
- | AND l_shipdate < date'1994-01-01' + interval 1 year
- | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01
- | AND l_quantity < 24;
+ | l_shipdate <= date '1998-12-01' - interval '90' day
+ |GROUP BY
+ | l_returnflag,
+ | l_linestatus
+ |ORDER BY
+ | l_returnflag,
+ | l_linestatus;
|""".stripMargin).show(200, false)
val tookTime = (System.nanoTime() - startTime) / 1000000
println(s"Execute ${i} time, time: ${tookTime}")
diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml
new file mode 100644
index 000000000000..53a944b5be2b
--- /dev/null
+++ b/backends-velox/pom.xml
@@ -0,0 +1,439 @@
+
+
+
+ gluten-parent
+ io.glutenproject
+ 1.0.0-snapshot
+
+ 4.0.0
+
+ backends-velox
+ jar
+ Gluten-Backends-Velox
+
+
+ 8
+ 8
+ ../cpp/
+ ../cpp/build/releases/
+ ${project.basedir}/src/main/resources
+ OFF
+ ${project.basedir}/../tools/build/arrow_install
+
+
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ provided
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ provided
+
+
+ org.apache.spark
+ spark-catalyst_${scala.binary.version}
+ provided
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ provided
+
+
+ org.apache.spark
+ spark-hive-thriftserver_${scala.binary.version}
+ provided
+
+
+ io.glutenproject
+ spark-sql-columnar-shims-common
+ ${project.version}
+ compile
+
+
+ io.glutenproject
+ gluten-jvm
+ ${project.version}
+ compile
+
+
+ org.scalacheck
+ scalacheck_${scala.binary.version}
+ 1.13.5
+ test
+
+
+ org.scala-lang
+ scala-library
+ ${scala.version}
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ test
+
+
+ org.mockito
+ mockito-core
+ 2.23.4
+ test
+
+
+ junit
+ junit
+ 4.12
+ test
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+ 2.10.0
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ 2.10.0
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ 2.10.0
+ provided
+
+
+ org.scalatestplus
+ scalatestplus-mockito_2.12
+ 1.0.0-M2
+ test
+
+
+ org.scalatestplus
+ scalatestplus-scalacheck_2.12
+ 3.1.0.0-RC2
+ test
+
+
+ org.apache.hadoop
+ hadoop-client
+ ${hadoop.version}
+ provided
+
+
+ commons-io
+ commons-io
+ 2.11.0
+ provided
+
+
+
+ org.apache.arrow
+ ${arrow-memory.artifact}
+ ${arrow.version}
+ runtime
+
+
+ org.apache.arrow
+ arrow-memory-core
+ ${arrow.version}
+
+
+ io.netty
+ netty-common
+
+
+ io.netty
+ netty-buffer
+
+
+
+
+
+ org.apache.arrow
+ arrow-vector
+ ${arrow.version}
+
+
+ io.netty
+ netty-common
+
+
+ io.netty
+ netty-buffer
+
+
+
+
+
+ org.apache.arrow.gandiva
+ arrow-gandiva
+ ${arrow.version}
+
+
+ io.netty
+ netty-common
+
+
+ io.netty
+ netty-buffer
+
+
+ protobuf-java
+ com.google.protobuf
+
+
+
+
+
+ org.apache.arrow
+ arrow-dataset
+ ${arrow.version}
+
+
+ io.netty
+ netty-common
+
+
+ io.netty
+ netty-buffer
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+
+
+ protobuf-java
+ com.google.protobuf
+
+
+ compile
+
+
+
+
+
+
+ ${cpp.build.dir}
+
+
+ ${resource.dir}
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
+ exec-maven-plugin
+ org.codehaus.mojo
+ 1.6.0
+
+
+ Build arrow
+ generate-resources
+
+ exec
+
+
+ bash
+
+ ${arrow.script.dir}/build_arrow.sh
+ --tests=${cpp_tests}
+ --build_arrow=${build_arrow}
+ --static_arrow=${static_arrow}
+ --arrow_root=${arrow_root}
+
+
+
+
+ Build cpp
+ generate-resources
+
+ exec
+
+
+ bash
+
+ ${cpp.dir}/compile.sh
+ ${build_cpp}
+ ${cpp_tests}
+ ${jvm.build_arrow}
+ ${static_arrow}
+ ${build_protobuf}
+ ${arrow_root}
+ ${jvm.arrow.bfs.install.dir}
+ ${build_jemalloc}
+ ${build_gazelle_cpp}
+ ${build_velox}
+ ${velox_home}
+
+
+
+
+ Build velox
+ generate-resources
+
+ exec
+
+
+ bash
+
+ ${velox.script.dir}/build_velox.sh
+ --build_velox_from_source=${build_velox_from_source}
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-resources-plugin
+ 3.0.1
+
+
+ net.alchim31.maven
+ scala-maven-plugin
+ 4.3.0
+
+ ${scala.recompile.mode}
+
+
+
+ scala-compile-first
+ process-resources
+
+ add-source
+ compile
+
+
+
+ scala-test-compile
+ process-test-resources
+
+ testCompile
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+ 3.8.0
+
+ 1.8
+ 1.8
+
+
+
+ compile
+
+ compile
+
+
+
+
+
+ maven-assembly-plugin
+ 3.3.0
+
+ ${jar.assembly.name.prefix}-${project.version}
+
+ jar-with-dependencies
+
+
+
+
+ make-assembly
+ package
+
+ single
+
+
+
+
+
+ org.scalastyle
+ scalastyle-maven-plugin
+ 1.0.0
+
+ false
+ false
+ true
+ false
+ ${project.basedir}/src/main/scala
+ ${project.basedir}/src/test/scala
+ ${project.basedir}/../scalastyle-config.xml
+ ${project.basedir}/scalastyle-output.xml
+ UTF-8
+
+
+
+
+ check
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-checkstyle-plugin
+ 3.1.1
+
+ false
+ true
+
+ ${project.basedir}/src/main/java
+ ${project.basedir}/src/main/scala
+
+
+ ${project.basedir}/src/test/java
+
+ ${project.basedir}/../dev/checkstyle.xml
+ ${project.basedir}/target/checkstyle-output.xml
+ UTF-8
+ UTF-8
+
+
+
+ com.puppycrawl.tools
+ checkstyle
+ 8.29
+
+
+
+
+
+ check
+
+
+
+
+
+ org.scalatest
+ scalatest-maven-plugin
+
+ true
+ .
+
+
+
+ test
+
+ test
+
+
+
+
+
+
+
\ No newline at end of file
diff --git a/jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java b/backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java
similarity index 57%
rename from jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java
rename to backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java
index 47be776ee895..6ba5b2236e2c 100644
--- a/jvm/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java
+++ b/backends-velox/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java
@@ -17,52 +17,24 @@
package io.glutenproject.execution;
-import io.glutenproject.expression.ConverterUtils;
+import io.glutenproject.expression.ArrowConverterUtils;
import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer;
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
import org.apache.spark.sql.vectorized.ColumnarBatch;
import java.util.Iterator;
-/**
- * Iterate on flatbuffers-serialized {@link org.apache.arrow.vector.ipc.message.ArrowRecordBatch}.
- *
- * {@link #next()} should be called from C++ scanner to read Java-generated Arrow data.
- */
-interface NativeSerializedRecordBatchIterator extends Iterator, AutoCloseable {
-
- /**
- * Return next serialized {@link org.apache.arrow.vector.ipc.message.ArrowRecordBatch} Java
- * byte array.
- */
- @Override
- byte[] next();
-}
-
-public class ColumnarNativeIterator implements NativeSerializedRecordBatchIterator {
- protected final Iterator delegated;
- protected ColumnarBatch nextBatch = null;
+public class ColumnarNativeIterator extends AbstractColumnarNativeIterator {
public ColumnarNativeIterator(Iterator delegated) {
- this.delegated = delegated;
- }
-
- @Override
- public boolean hasNext() {
- while (delegated.hasNext()) {
- nextBatch = delegated.next();
- if (nextBatch.numRows() > 0) {
- return true;
- }
- }
- return false;
+ super(delegated);
}
@Override
public byte[] next() {
ColumnarBatch dep_cb = nextBatch;
if (dep_cb.numRows() > 0) {
- ArrowRecordBatch dep_rb = ConverterUtils.createArrowRecordBatch(dep_cb);
+ ArrowRecordBatch dep_rb = ArrowConverterUtils.createArrowRecordBatch(dep_cb);
return serialize(dep_rb);
} else {
throw new IllegalStateException();
@@ -72,9 +44,4 @@ public byte[] next() {
private byte[] serialize(ArrowRecordBatch batch) {
return UnsafeRecordBatchSerializer.serializeUnsafe(batch);
}
-
- @Override
- public void close() throws Exception {
-
- }
}
diff --git a/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala b/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala
new file mode 100644
index 000000000000..5b2929f94aa5
--- /dev/null
+++ b/backends-velox/src/main/java/io/glutenproject/execution/VeloxNativeColumnarToRowExec.scala
@@ -0,0 +1,178 @@
+/*
+ * 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 io.glutenproject.execution
+
+import io.glutenproject.GlutenConfig
+import io.glutenproject.expression.ArrowConverterUtils
+import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativeColumnarToRowJniWrapper}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+import scala.concurrent.duration._
+
+import org.apache.arrow.vector.types.pojo.{Field, Schema}
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
+import org.apache.spark.sql.types._
+
+class VeloxNativeColumnarToRowExec(child: SparkPlan)
+ extends NativeColumnarToRowExec(child = child) {
+ override def nodeName: String = "VeloxNativeColumnarToRowExec"
+
+ override def supportCodegen: Boolean = false
+
+ override def buildCheck(): Unit = {
+ val schema = child.schema
+ for (field <- schema.fields) {
+ field.dataType match {
+ case d: BooleanType =>
+ case d: ByteType =>
+ case d: ShortType =>
+ case d: IntegerType =>
+ case d: LongType =>
+ case d: FloatType =>
+ case d: DoubleType =>
+ case d: StringType =>
+ case d: DateType =>
+ case d: DecimalType =>
+ case d: TimestampType =>
+ case d: BinaryType =>
+ case _ =>
+ throw new UnsupportedOperationException(s"${field.dataType} is not supported in NativeColumnarToRowExec.")
+ }
+ }
+ }
+
+ override def doExecuteInternal(): RDD[InternalRow] = {
+ val numOutputRows = longMetric("numOutputRows")
+ val numInputBatches = longMetric("numInputBatches")
+ val convertTime = longMetric("convertTime")
+
+ // Fake Arrow format will be returned for WS transformer.
+ // TODO: use a Velox layer on the top of the base layer.
+ if (GlutenConfig.getConf.isVeloxBackend) {
+ if (child.isInstanceOf[WholeStageTransformerExec]) {
+ child.asInstanceOf[WholeStageTransformerExec].setFakeOutput()
+ }
+ }
+
+ child.executeColumnar().mapPartitions { batches =>
+ // TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast
+ val jniWrapper = new NativeColumnarToRowJniWrapper()
+ var arrowSchema: Array[Byte] = null
+
+ def serializeSchema(fields: Seq[Field]): Array[Byte] = {
+ val schema = new Schema(fields.asJava)
+ ArrowConverterUtils.getSchemaBytesBuf(schema)
+ }
+
+ batches.flatMap { batch =>
+ numInputBatches += 1
+ numOutputRows += batch.numRows()
+
+ if (batch.numRows == 0) {
+ logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols")
+ Iterator.empty
+ } else if (this.output.isEmpty || (batch.numCols() > 0 &&
+ !batch.column(0).isInstanceOf[ArrowWritableColumnVector])) {
+ // Fallback to ColumnarToRow
+ val localOutput = this.output
+ numInputBatches += 1
+ numOutputRows += batch.numRows()
+
+ val toUnsafe = UnsafeProjection.create(localOutput, localOutput)
+ batch.rowIterator().asScala.map(toUnsafe)
+ } else {
+ val bufAddrs = new ListBuffer[Long]()
+ val bufSizes = new ListBuffer[Long]()
+ val fields = new ListBuffer[Field]()
+ (0 until batch.numCols).foreach { idx =>
+ val column = batch.column(idx).asInstanceOf[ArrowWritableColumnVector]
+ fields += column.getValueVector.getField
+ column.getValueVector.getBuffers(false)
+ .foreach { buffer =>
+ bufAddrs += buffer.memoryAddress()
+ try {
+ bufSizes += buffer.readableBytes()
+ } catch {
+ case e: Throwable =>
+ // For Velox, the returned format is faked arrow format,
+ // and the offset buffer is invalid. Only the buffer address is cared.
+ if (GlutenConfig.getConf.isVeloxBackend &&
+ child.output(idx).dataType == StringType) {
+ // Add a fake value here for String column.
+ bufSizes += 1
+ } else {
+ throw e
+ }
+ }
+ }
+ }
+
+ if (arrowSchema == null) {
+ arrowSchema = serializeSchema(fields)
+ }
+
+ val beforeConvert = System.nanoTime()
+
+ val info = jniWrapper.nativeConvertColumnarToRow(
+ arrowSchema, batch.numRows, bufAddrs.toArray, bufSizes.toArray,
+ SparkMemoryUtils.contextMemoryPool().getNativeInstanceId, wsChild)
+
+ convertTime += NANOSECONDS.toMillis(System.nanoTime() - beforeConvert)
+
+ new Iterator[InternalRow] {
+ var rowId = 0
+ val row = new UnsafeRow(batch.numCols())
+ var closed = false
+ override def hasNext: Boolean = {
+ val result = rowId < batch.numRows()
+ if (!result && !closed) {
+ jniWrapper.nativeClose(info.instanceID)
+ closed = true
+ }
+ return result
+ }
+
+ override def next: UnsafeRow = {
+ if (rowId >= batch.numRows()) throw new NoSuchElementException
+
+ val (offset, length) = (info.offsets(rowId), info.lengths(rowId))
+ row.pointTo(null, info.memoryAddress + offset, length.toInt)
+ rowId += 1
+ row
+ }
+ }
+ }
+ }
+ }
+ }
+
+ override def canEqual(other: Any): Boolean = other.isInstanceOf[VeloxNativeColumnarToRowExec]
+
+ override def equals(other: Any): Boolean = other match {
+ case that: VeloxNativeColumnarToRowExec =>
+ (that canEqual this) && super.equals(that)
+ case _ => false
+ }
+}
+
diff --git a/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala b/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala
new file mode 100644
index 000000000000..4178d4866f01
--- /dev/null
+++ b/backends-velox/src/main/java/io/glutenproject/execution/VeloxRowToArrowColumnarExec.scala
@@ -0,0 +1,293 @@
+/*
+ * 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 io.glutenproject.execution
+
+import java.util.concurrent.TimeUnit._
+
+import io.glutenproject.vectorized._
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.SpecializedGetters
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+class RowToColumnConverter(schema: StructType) extends Serializable {
+ private val converters = schema.fields.map {
+ f => RowToColumnConverter.getConverterForType(f.dataType, f.nullable)
+ }
+
+ final def convert(row: InternalRow, vectors: Array[WritableColumnVector]): Unit = {
+ var idx = 0
+ while (idx < row.numFields) {
+ converters(idx).append(row, idx, vectors(idx))
+ idx += 1
+ }
+ }
+}
+
+object RowToColumnConverter {
+ private abstract class TypeConverter extends Serializable {
+ def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit
+ }
+
+ private final case class BasicNullableTypeConverter(base: TypeConverter) extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ if (row.isNullAt(column)) {
+ cv.appendNull
+ } else {
+ base.append(row, column, cv)
+ }
+ }
+ }
+
+ private final case class StructNullableTypeConverter(base: TypeConverter) extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ if (row.isNullAt(column)) {
+ cv.appendStruct(true)
+ } else {
+ base.append(row, column, cv)
+ }
+ }
+ }
+
+ private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = {
+ val core = dataType match {
+ case BooleanType => BooleanConverter
+ case ByteType => ByteConverter
+ case ShortType => ShortConverter
+ case IntegerType | DateType => IntConverter
+ case FloatType => FloatConverter
+ case LongType | TimestampType => LongConverter
+ case DoubleType => DoubleConverter
+ case StringType => StringConverter
+ case BinaryType => BinaryConverter
+ case CalendarIntervalType => CalendarConverter
+ case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable))
+ case st: StructType => new StructConverter(st.fields.map(
+ (f) => getConverterForType(f.dataType, f.nullable)))
+ case dt: DecimalType => new DecimalConverter(dt)
+ case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable),
+ getConverterForType(mt.valueType, nullable))
+ case unknown => throw new UnsupportedOperationException(
+ s"Type $unknown not supported")
+ }
+
+ if (nullable) {
+ dataType match {
+ case CalendarIntervalType => new StructNullableTypeConverter(core)
+ case st: StructType => new StructNullableTypeConverter(core)
+ case _ => new BasicNullableTypeConverter(core)
+ }
+ } else {
+ core
+ }
+ }
+
+ private object BooleanConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendBoolean(row.getBoolean(column))
+ }
+
+ private object ByteConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendByte(row.getByte(column))
+ }
+
+ private object ShortConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendShort(row.getShort(column))
+ }
+
+ private object IntConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendInt(row.getInt(column))
+ }
+
+ private object FloatConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendFloat(row.getFloat(column))
+ }
+
+ private object LongConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendLong(row.getLong(column))
+ }
+
+ private object DoubleConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
+ cv.appendDouble(row.getDouble(column))
+ }
+
+ private object StringConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val data = row.getUTF8String(column).getBytes
+ cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length)
+ }
+ }
+
+ private object BinaryConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val data = row.getBinary(column)
+ cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length)
+ }
+ }
+
+ private object CalendarConverter extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val c = row.getInterval(column)
+ cv.appendStruct(false)
+ cv.getChild(0).appendInt(c.months)
+ cv.getChild(1).appendInt(c.days)
+ cv.getChild(2).appendLong(c.microseconds)
+ }
+ }
+
+ private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val values = row.getArray(column)
+ val numElements = values.numElements()
+ cv.appendArray(numElements)
+ val arrData = cv.arrayData()
+ for (i <- 0 until numElements) {
+ childConverter.append(values, i, arrData)
+ }
+ }
+ }
+
+ private case class StructConverter(childConverters: Array[TypeConverter]) extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ cv.appendStruct(false)
+ val data = row.getStruct(column, childConverters.length)
+ for (i <- 0 until childConverters.length) {
+ childConverters(i).append(data, i, cv.getChild(i))
+ }
+ }
+ }
+
+ private case class DecimalConverter(dt: DecimalType) extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val d = row.getDecimal(column, dt.precision, dt.scale)
+ if (dt.precision <= Decimal.MAX_INT_DIGITS) {
+ cv.appendInt(d.toUnscaledLong.toInt)
+ } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) {
+ cv.appendLong(d.toUnscaledLong)
+ } else {
+ val value = d.toJavaBigDecimal
+ cv.asInstanceOf[ArrowWritableColumnVector].appendDecimal(value)
+ }
+ }
+ }
+
+ private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter)
+ extends TypeConverter {
+ override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
+ val m = row.getMap(column)
+ val keys = cv.getChild(0)
+ val values = cv.getChild(1)
+ val numElements = m.numElements()
+ cv.appendArray(numElements)
+
+ val srcKeys = m.keyArray()
+ val srcValues = m.valueArray()
+
+ for (i <- 0 until numElements) {
+ keyConverter.append(srcKeys, i, keys)
+ valueConverter.append(srcValues, i, values)
+ }
+ }
+ }
+}
+
+/**
+ * Provides a common executor to translate an [[RDD]] of [[InternalRow]] into an [[RDD]] of
+ * [[ColumnarBatch]]. This is inserted whenever such a transition is determined to be needed.
+ *
+ * This is similar to some of the code in ArrowConverters.scala and
+ * [[org.apache.spark.sql.execution.arrow.ArrowWriter]]. That code is more specialized
+ * to convert [[InternalRow]] to Arrow formatted data, but in the future if we make
+ * [[OffHeapColumnVector]] internally Arrow formatted we may be able to replace much of that code.
+ *
+ * This is also similar to
+ * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate()]] and
+ * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.toBatch()]] toBatch is only ever
+ * called from tests and can probably be removed, but populate is used by both Orc and Parquet
+ * to initialize partition and missing columns. There is some chance that we could replace
+ * populate with [[RowToColumnConverter]], but the performance requirements are different and it
+ * would only be to reduce code.
+ */
+class VeloxRowToArrowColumnarExec(child: SparkPlan) extends RowToArrowColumnarExec(child = child) {
+
+ override def doExecuteColumnarInternal(): RDD[ColumnarBatch] = {
+ val numInputRows = longMetric("numInputRows")
+ val numOutputBatches = longMetric("numOutputBatches")
+ val processTime = longMetric("processTime")
+ // Instead of creating a new config we are reusing columnBatchSize. In the future if we do
+ // combine with some of the Arrow conversion tools we will need to unify some of the configs.
+ val numRows = conf.columnBatchSize
+ // This avoids calling `schema` in the RDD closure, so that we don't need to include the entire
+ // plan (this) in the closure.
+ val localSchema = this.schema
+ child.execute().mapPartitions { rowIterator =>
+ if (rowIterator.hasNext) {
+ val res = new Iterator[ColumnarBatch] {
+ private val converters = new RowToColumnConverter(localSchema)
+ private var last_cb: ColumnarBatch = null
+ private var elapse: Long = 0
+
+ override def hasNext: Boolean = {
+ rowIterator.hasNext
+ }
+
+ override def next(): ColumnarBatch = {
+ val vectors: Seq[WritableColumnVector] =
+ ArrowWritableColumnVector.allocateColumns(numRows, schema)
+ var rowCount = 0
+ while (rowCount < numRows && rowIterator.hasNext) {
+ val row = rowIterator.next()
+ val start = System.nanoTime()
+ converters.convert(row, vectors.toArray)
+ elapse += System.nanoTime() - start
+ rowCount += 1
+ }
+ vectors.foreach(v => v.asInstanceOf[ArrowWritableColumnVector].setValueCount(rowCount))
+ processTime.set(NANOSECONDS.toMillis(elapse))
+ numInputRows += rowCount
+ numOutputBatches += 1
+ last_cb = new ColumnarBatch(vectors.toArray, rowCount)
+ last_cb
+ }
+ }
+ new UnsafeItr(res)
+ } else {
+ Iterator.empty
+ }
+ }
+ }
+
+ override def canEqual(other: Any): Boolean = other.isInstanceOf[VeloxRowToArrowColumnarExec]
+
+ override def equals(other: Any): Boolean = other match {
+ case that: VeloxRowToArrowColumnarExec =>
+ (that canEqual this) && super.equals(that)
+ case _ => false
+ }
+}
diff --git a/jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java b/backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java
similarity index 86%
rename from jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java
rename to backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java
index a52e131dba5c..1cb30c8a04f4 100644
--- a/jvm/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java
+++ b/backends-velox/src/main/java/io/glutenproject/expression/ArrowColumnarBatch.java
@@ -17,14 +17,11 @@
package io.glutenproject.expression;
-import java.io.*;
-import org.apache.arrow.memory.ArrowBuf;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
import org.apache.arrow.vector.ValueVector;
+import java.io.IOException;
+import java.util.List;
+
public class ArrowColumnarBatch implements AutoCloseable {
int fieldsNum;
diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java
rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryConsumer.java
diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java
rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/NativeSQLMemoryMetrics.java
diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java
rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedAllocationListener.java
diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java
rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/SparkManagedReservationListener.java
diff --git a/jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java b/backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java
rename to backends-velox/src/main/java/io/glutenproject/spark/sql/execution/datasources/v2/arrow/Spiller.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowBufBuilder.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowCompressedStreamReader.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowFieldNodeBuilder.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ArrowWritableColumnVector.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java b/backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java
similarity index 61%
rename from jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java
index 4cc5866e8e02..4699f46b7ed1 100644
--- a/jvm/src/main/java/io/glutenproject/vectorized/BatchIterator.java
+++ b/backends-velox/src/main/java/io/glutenproject/vectorized/BatchIterator.java
@@ -17,44 +17,37 @@
package io.glutenproject.vectorized;
-import java.io.IOException;
-
import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
-import org.apache.arrow.vector.types.pojo.Schema;
-
-import java.io.Serializable;
-import java.io.ByteArrayOutputStream;
-import java.nio.channels.Channels;
-import org.apache.arrow.vector.ipc.WriteChannel;
-import org.apache.arrow.vector.ipc.message.MessageSerializer;
import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.vectorized.ColumnVector;
import org.apache.spark.sql.vectorized.ColumnarBatch;
-public class BatchIterator implements AutoCloseable, Serializable {
+import java.io.IOException;
+import java.io.Serializable;
+
+public class BatchIterator extends AbstractBatchIterator {
private native boolean nativeHasNext(long nativeHandler);
private native byte[] nativeNext(long nativeHandler);
private native long nativeCHNext(long nativeHandler);
private native void nativeClose(long nativeHandler);
private native MetricsObject nativeFetchMetrics(long nativeHandler);
- private long nativeHandler = 0;
- private boolean closed = false;
-
public BatchIterator() throws IOException {}
public BatchIterator(long instance_id) throws IOException {
- nativeHandler = instance_id;
+ super(instance_id);
}
- public boolean hasNext() throws IOException {
+ @Override
+ public boolean hasNextInternal() throws IOException {
return nativeHasNext(nativeHandler);
}
- public ArrowRecordBatch next() throws IOException {
+ @Override
+ public ArrowRecordBatch nextInternal() throws IOException {
BufferAllocator allocator = SparkMemoryUtils.contextAllocator();
if (nativeHandler == 0) {
return null;
@@ -67,39 +60,16 @@ public ArrowRecordBatch next() throws IOException {
serializedRecordBatch);
}
- public MetricsObject getMetrics() throws IOException, ClassNotFoundException {
- if (nativeHandler == 0) {
- return null;
- }
+ @Override
+ public MetricsObject getMetricsInternal() throws IOException, ClassNotFoundException {
return nativeFetchMetrics(nativeHandler);
}
- public ColumnarBatch chNext() {
- long block = nativeCHNext(nativeHandler);
- CHNativeBlock nativeBlock = new CHNativeBlock(block);
- int cols = nativeBlock.numColumns();
- ColumnVector[] columnVectors = new ColumnVector[cols];
- for (int i = 0; i < cols; i++) {
- columnVectors[i] = new CHColumnVector(DataType.fromDDL(nativeBlock.getTypeByPosition(i)), block, i);
- }
- return new ColumnarBatch(columnVectors, nativeBlock.numRows());
- }
-
@Override
- public void close() {
+ public void closeInternal() {
if (!closed) {
nativeClose(nativeHandler);
closed = true;
}
}
-
- byte[] getSchemaBytesBuf(Schema schema) throws IOException {
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema);
- return out.toByteArray();
- }
-
- long getInstanceId() {
- return nativeHandler;
- }
}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java
similarity index 99%
rename from jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java
index 18707af95a7d..8f6288f145ec 100644
--- a/jvm/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java
+++ b/backends-velox/src/main/java/io/glutenproject/vectorized/CompressedVectorLoader.java
@@ -17,8 +17,7 @@
package io.glutenproject.vectorized;
-import java.util.Iterator;
-
+import org.apache.arrow.memory.ArrowBuf;
import org.apache.arrow.util.Collections2;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.VectorLoader;
@@ -26,7 +25,7 @@
import org.apache.arrow.vector.ipc.message.ArrowFieldNode;
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
-import org.apache.arrow.memory.ArrowBuf;
+import java.util.Iterator;
/** Loads compressed buffers into vectors. */
public class CompressedVectorLoader extends VectorLoader {
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowCompressedStreamReader.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/SchemaAwareArrowStreamReader.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java b/backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java
similarity index 97%
rename from jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java
index 9084eb41880e..b96d17f8d776 100644
--- a/jvm/src/main/java/io/glutenproject/vectorized/SerializableObject.java
+++ b/backends-velox/src/main/java/io/glutenproject/vectorized/SerializableObject.java
@@ -17,7 +17,6 @@
package io.glutenproject.vectorized;
-import java.io.*;
import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.KryoSerializable;
import com.esotericsoftware.kryo.io.Input;
@@ -30,6 +29,12 @@
import io.netty.buffer.UnpooledByteBufAllocator;
import org.apache.arrow.util.AutoCloseables;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.OutputStream;
+
/** ArrowBufBuilder. */
public class SerializableObject implements Externalizable, KryoSerializable {
public int total_size;
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleDecompressionJniWrapper.java
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java b/backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java
similarity index 100%
rename from jvm/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java
rename to backends-velox/src/main/java/io/glutenproject/vectorized/ShuffleSplitterJniWrapper.java
diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi
new file mode 100644
index 000000000000..afdecf215121
--- /dev/null
+++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.velox.VeloxIteratorApi
\ No newline at end of file
diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi
new file mode 100644
index 000000000000..b11a3a9bbe4e
--- /dev/null
+++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.velox.VeloxSparkPlanExecApi
\ No newline at end of file
diff --git a/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi
new file mode 100644
index 000000000000..0716759ad1e9
--- /dev/null
+++ b/backends-velox/src/main/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.velox.VeloxTransformerApi
\ No newline at end of file
diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala
new file mode 100644
index 000000000000..c74f3b392075
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxIteratorApi.scala
@@ -0,0 +1,381 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi.velox
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import io.glutenproject.{GlutenConfig, GlutenNumaBindingInfo}
+import io.glutenproject.backendsapi.IIteratorApi
+import io.glutenproject.execution._
+import io.glutenproject.expression.ArrowConverterUtils
+import io.glutenproject.substrait.plan.PlanNode
+import io.glutenproject.substrait.rel.LocalFilesBuilder
+import io.glutenproject.vectorized._
+import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
+import org.apache.arrow.vector.types.pojo.Schema
+import org.apache.spark.{InterruptibleIterator, SparkConf, TaskContext}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.FilePartition
+import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
+import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils}
+
+class VeloxIteratorApi extends IIteratorApi with Logging {
+
+ /**
+ * Generate native row partition.
+ *
+ * @return
+ */
+ override def genNativeFilePartition(p: InputPartition,
+ wsCxt: WholestageTransformContext
+ ): BaseNativeFilePartition = {
+ p match {
+ case FilePartition(index, files) =>
+ val paths = new java.util.ArrayList[String]()
+ val starts = new java.util.ArrayList[java.lang.Long]()
+ val lengths = new java.util.ArrayList[java.lang.Long]()
+ files.foreach { f =>
+ paths.add(f.filePath)
+ starts.add(new java.lang.Long(f.start))
+ lengths.add(new java.lang.Long(f.length))
+ }
+ val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths)
+ wsCxt.substraitContext.setLocalFilesNode(localFilesNode)
+ val substraitPlan = wsCxt.root.toProtobuf
+ /*
+ val out = new DataOutputStream(new FileOutputStream("/tmp/SubStraitTest-Q6.dat",
+ false));
+ out.write(substraitPlan.toByteArray());
+ out.flush();
+ */
+ // logWarning(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}")
+ NativeFilePartition(index, files, substraitPlan.toByteArray)
+ }
+ }
+
+
+ /**
+ * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec.
+ *
+ * @param iter
+ * @param recordsPerBatch
+ * @param numOutputRows
+ * @param numInputBatches
+ * @param numOutputBatches
+ * @param collectTime
+ * @param concatTime
+ * @param avgCoalescedNumRows
+ * @return
+ */
+ override def genCoalesceIterator(iter: Iterator[ColumnarBatch],
+ recordsPerBatch: Int,
+ numOutputRows: SQLMetric,
+ numInputBatches: SQLMetric,
+ numOutputBatches: SQLMetric,
+ collectTime: SQLMetric,
+ concatTime: SQLMetric,
+ avgCoalescedNumRows: SQLMetric): Iterator[ColumnarBatch] = {
+ import io.glutenproject.utils.VeloxImplicitClass._
+
+ val beforeInput = System.nanoTime
+ val hasInput = iter.hasNext
+ collectTime += System.nanoTime - beforeInput
+ val res = if (hasInput) {
+ new Iterator[ColumnarBatch] {
+ var numBatchesTotal: Long = _
+ var numRowsTotal: Long = _
+ SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ =>
+ if (numBatchesTotal > 0) {
+ avgCoalescedNumRows.set(numRowsTotal.toDouble / numBatchesTotal)
+ }
+ }
+
+ override def hasNext: Boolean = {
+ val beforeNext = System.nanoTime
+ val hasNext = iter.hasNext
+ collectTime += System.nanoTime - beforeNext
+ hasNext
+ }
+
+ override def next(): ColumnarBatch = {
+ if (!hasNext) {
+ throw new NoSuchElementException("End of ColumnarBatch iterator")
+ }
+
+ var rowCount = 0
+ val batchesToAppend = ListBuffer[ColumnarBatch]()
+
+ while (hasNext && rowCount < recordsPerBatch) {
+ val delta = iter.next()
+ delta.retain()
+ rowCount += delta.numRows
+ batchesToAppend += delta
+ }
+
+ // chendi: We need make sure target FieldTypes are exactly the same as src
+ val expected_output_arrow_fields = if (batchesToAppend.size > 0) {
+ (0 until batchesToAppend(0).numCols).map(i => {
+ batchesToAppend(0).column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector.getField
+ })
+ } else {
+ Nil
+ }
+
+ val resultStructType = ArrowUtils.fromArrowSchema(new Schema(expected_output_arrow_fields.asJava))
+ val beforeConcat = System.nanoTime
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(rowCount, resultStructType).toArray
+ val target =
+ new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), rowCount)
+ coalesce(target, batchesToAppend.toList)
+ target.setNumRows(rowCount)
+
+ concatTime += System.nanoTime - beforeConcat
+ numOutputRows += rowCount
+ numInputBatches += batchesToAppend.length
+ numOutputBatches += 1
+
+ // used for calculating avgCoalescedNumRows
+ numRowsTotal += rowCount
+ numBatchesTotal += 1
+
+ batchesToAppend.foreach(cb => cb.close())
+
+ target
+ }
+ }
+ } else {
+ Iterator.empty
+ }
+ new CloseableColumnBatchIterator(res)
+ }
+
+
+ /**
+ * Generate closeable ColumnBatch iterator.
+ *
+ * @param iter
+ * @return
+ */
+ override def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]
+ ): Iterator[ColumnarBatch] = {
+ new CloseableColumnBatchIterator(iter)
+ }
+
+ /**
+ * Generate Iterator[ColumnarBatch] for first stage.
+ *
+ * @return
+ */
+ override def genFirstStageIterator(inputPartition: BaseNativeFilePartition,
+ loadNative: Boolean,
+ outputAttributes: Seq[Attribute],
+ context: TaskContext,
+ jarList: Seq[String]): Iterator[ColumnarBatch] = {
+ import org.apache.spark.sql.util.OASPackageBridge._
+ var inputSchema : Schema = null
+ var outputSchema : Schema = null
+ var resIter : AbstractBatchIterator = null
+ if (loadNative) {
+ // TODO: 'jarList' is kept for codegen
+ val transKernel = new ExpressionEvaluator(jarList.asJava)
+ val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]()
+ outputSchema = ArrowConverterUtils.toArrowSchema(outputAttributes)
+ resIter = transKernel.createKernelWithBatchIterator(
+ inputPartition.substraitPlan, inBatchIters)
+ SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => resIter.close() }
+ }
+ val iter = new Iterator[Any] {
+ private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics
+
+ override def hasNext: Boolean = {
+ if (loadNative) {
+ resIter.hasNext
+ } else {
+ false
+ }
+ }
+
+ override def next(): Any = {
+ if (!hasNext) {
+ throw new java.util.NoSuchElementException("End of stream")
+ }
+ val rb = resIter.next().asInstanceOf[ArrowRecordBatch]
+ if (rb == null) {
+ val resultStructType = ArrowUtils.fromArrowSchema(outputSchema)
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
+ return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
+ }
+ val outputNumRows = rb.getLength
+ val output = ArrowConverterUtils.fromArrowRecordBatch(outputSchema, rb)
+ ArrowConverterUtils.releaseArrowRecordBatch(rb)
+ val cb = new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows)
+ val bytes: Long = cb match {
+ case batch: ColumnarBatch =>
+ (0 until batch.numCols()).map { i =>
+ val vector = Option(batch.column(i))
+ vector.map {
+ case av: ArrowWritableColumnVector =>
+ av.getValueVector.getBufferSize.toLong
+ case _ => 0L
+ }.sum
+ }.sum
+ case _ => 0L
+ }
+ inputMetrics.bridgeIncBytesRead(bytes)
+ cb
+ }
+ }
+
+ // TODO: SPARK-25083 remove the type erasure hack in data source scan
+ new InterruptibleIterator(context,
+ new CloseableColumnBatchIterator(iter.asInstanceOf[Iterator[ColumnarBatch]]))
+ }
+
+ /**
+ * Generate Iterator[ColumnarBatch] for final stage.
+ *
+ * @return
+ */
+ override def genFinalStageIterator(iter: Iterator[ColumnarBatch],
+ numaBindingInfo: GlutenNumaBindingInfo,
+ listJars: Seq[String],
+ signature: String,
+ sparkConf: SparkConf,
+ outputAttributes: Seq[Attribute],
+ rootNode: PlanNode,
+ streamedSortPlan: SparkPlan,
+ pipelineTime: SQLMetric,
+ buildRelationBatchHolder: Seq[ColumnarBatch],
+ dependentKernels: Seq[ExpressionEvaluator],
+ dependentKernelIterators: Seq[AbstractBatchIterator]
+ ): Iterator[ColumnarBatch] = {
+ ExecutorManager.tryTaskSet(numaBindingInfo)
+ GlutenConfig.getConf
+ var build_elapse: Long = 0
+ var eval_elapse: Long = 0
+ val execTempDir = GlutenConfig.getTempFile
+ val jarList = listJars.map(jarUrl => {
+ logWarning(s"Get Codegened library Jar ${jarUrl}")
+ UserAddedJarUtils.fetchJarFromSpark(
+ jarUrl,
+ execTempDir,
+ s"spark-columnar-plugin-codegen-precompile-${signature}.jar",
+ sparkConf)
+ s"${execTempDir}/spark-columnar-plugin-codegen-precompile-${signature}.jar"
+ })
+
+ val transKernel = new ExpressionEvaluator(jarList.toList.asJava)
+ val inBatchIter = new ColumnarNativeIterator(iter.asJava)
+ val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]()
+ inBatchIters.add(inBatchIter)
+ // we need to complete dependency RDD's firstly
+ val beforeBuild = System.nanoTime()
+ val outputSchema = ArrowConverterUtils.toArrowSchema(outputAttributes)
+ val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters)
+ build_elapse += System.nanoTime() - beforeBuild
+ val resultStructType = ArrowUtils.fromArrowSchema(outputSchema)
+ val resIter = streamedSortPlan match {
+ case t: TransformSupport =>
+ new Iterator[ColumnarBatch] {
+ override def hasNext: Boolean = {
+ val res = nativeIterator.hasNext
+ // if (res == false) updateMetrics(nativeIterator)
+ res
+ }
+
+ override def next(): ColumnarBatch = {
+ val beforeEval = System.nanoTime()
+ val output_rb = nativeIterator.next.asInstanceOf[ArrowRecordBatch]
+ if (output_rb == null) {
+ eval_elapse += System.nanoTime() - beforeEval
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
+ return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
+ }
+ val outputNumRows = output_rb.getLength
+ val outSchema = ArrowConverterUtils.toArrowSchema(outputAttributes)
+ val output = ArrowConverterUtils.fromArrowRecordBatch(outSchema, output_rb)
+ ArrowConverterUtils.releaseArrowRecordBatch(output_rb)
+ eval_elapse += System.nanoTime() - beforeEval
+ new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows)
+ }
+ }
+ case _ =>
+ throw new UnsupportedOperationException(
+ s"streamedSortPlan should support transformation")
+ }
+ var closed = false
+
+ def close = {
+ closed = true
+ pipelineTime += (eval_elapse + build_elapse) / 1000000
+ buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative
+ dependentKernels.foreach(_.close)
+ dependentKernelIterators.foreach(_.close)
+ nativeIterator.close()
+ // relationHolder.clear()
+ }
+
+ SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => {
+ close
+ })
+ new CloseableColumnBatchIterator(resIter)
+ }
+
+ /**
+ * Generate columnar native iterator.
+ *
+ * @return
+ */
+ override def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch]
+ ): ColumnarNativeIterator = {
+ new ColumnarNativeIterator(delegated.asJava)
+ }
+
+ /**
+ * Generate BatchIterator for ExpressionEvaluator.
+ *
+ * @return
+ */
+ override def genBatchIterator(wsPlan: Array[Byte],
+ iterList: Seq[AbstractColumnarNativeIterator],
+ jniWrapper: ExpressionEvaluatorJniWrapper
+ ): AbstractBatchIterator = {
+ val memoryPool = SparkMemoryUtils.contextMemoryPool();
+ val poolId = memoryPool.getNativeInstanceId();
+ val batchIteratorInstance = jniWrapper.nativeCreateKernelWithIterator(
+ poolId, wsPlan, iterList.toArray);
+ new BatchIterator(batchIteratorInstance)
+ }
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND
+}
diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala
new file mode 100644
index 000000000000..9d4d05929dde
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxSparkPlanExecApi.scala
@@ -0,0 +1,110 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi.velox
+
+import scala.collection.JavaConverters._
+
+import io.glutenproject.backendsapi.ISparkPlanExecApi
+import io.glutenproject.GlutenConfig
+import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec, VeloxNativeColumnarToRowExec, VeloxRowToArrowColumnarExec}
+import io.glutenproject.vectorized.ArrowColumnarBatchSerializer
+import org.apache.spark.ShuffleDependency
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
+import org.apache.spark.shuffle.utils.VeloxShuffleUtil
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.utils.VeloxExecUtil
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+class VeloxSparkPlanExecApi extends ISparkPlanExecApi {
+
+ /**
+ * Generate NativeColumnarToRowExec.
+ *
+ * @param child
+ * @return
+ */
+ override def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec =
+ new VeloxNativeColumnarToRowExec(child)
+
+ /**
+ * Generate RowToArrowColumnarExec.
+ *
+ * @param child
+ * @return
+ */
+ override def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec =
+ new VeloxRowToArrowColumnarExec(child)
+
+ /**
+ * Generate ShuffleDependency for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def genShuffleDependency(rdd: RDD[ColumnarBatch],
+ outputAttributes: Seq[Attribute],
+ newPartitioning: Partitioning,
+ serializer: Serializer,
+ writeMetrics: Map[String, SQLMetric],
+ dataSize: SQLMetric,
+ bytesSpilled: SQLMetric,
+ numInputRows: SQLMetric,
+ computePidTime: SQLMetric,
+ splitTime: SQLMetric,
+ spillTime: SQLMetric,
+ compressTime: SQLMetric,
+ prepareTime: SQLMetric
+ ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = {
+ VeloxExecUtil.genShuffleDependency(rdd, outputAttributes, newPartitioning,
+ serializer, writeMetrics, dataSize, bytesSpilled, numInputRows,
+ computePidTime, splitTime, spillTime, compressTime, prepareTime)
+ }
+
+ /**
+ * Generate ColumnarShuffleWriter for ColumnarShuffleManager.
+ *
+ * @return
+ */
+ override def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V]
+ ): GlutenShuffleWriterWrapper[K, V] = {
+ VeloxShuffleUtil.genColumnarShuffleWriter(parameters)
+ }
+
+ /**
+ * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def createColumnarBatchSerializer(schema: StructType,
+ readBatchNumRows: SQLMetric,
+ numOutputRows: SQLMetric): Serializer = {
+ new ArrowColumnarBatchSerializer(schema, readBatchNumRows, numOutputRows)
+ }
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND
+}
diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala
new file mode 100644
index 000000000000..149f572a806a
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxTransformerApi.scala
@@ -0,0 +1,65 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi.velox
+
+import scala.collection.JavaConverters._
+
+import io.glutenproject.backendsapi.ITransformerApi
+import io.glutenproject.GlutenConfig
+import io.glutenproject.expression.ArrowConverterUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning}
+
+class VeloxTransformerApi extends ITransformerApi with Logging {
+
+ /**
+ * Do validate for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning,
+ outputAttributes: Seq[Attribute]
+ ): Boolean = {
+ // check input datatype
+ for (attr <- outputAttributes) {
+ try ArrowConverterUtils.createArrowField(attr) catch {
+ case e: UnsupportedOperationException =>
+ logInfo(s"${attr.dataType} is not supported in VeloxColumnarShuffledExchangeExec.")
+ return false
+ }
+ }
+ outputPartitioning match {
+ case HashPartitioning(exprs, n) =>
+ exprs.foreach(expr => {
+ if (!expr.isInstanceOf[Attribute]) {
+ logInfo("Expressions are not supported in HashPartitioning.")
+ return false
+ }})
+ case _ =>
+ }
+ true
+ }
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = GlutenConfig.GLUTEN_VELOX_BACKEND
+}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala b/backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala
similarity index 83%
rename from jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala
rename to backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala
index 33866e092bef..f4106e8d560e 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala
+++ b/backends-velox/src/main/scala/io/glutenproject/execution/BroadcastColumnarRDD.scala
@@ -18,6 +18,7 @@
package io.glutenproject.execution
import io.glutenproject.vectorized.CloseableColumnBatchIterator
+
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.execution._
@@ -27,11 +28,11 @@ import org.apache.spark.sql.vectorized.ColumnarBatch
private final case class BroadcastColumnarRDDPartition(index: Int) extends Partition
case class BroadcastColumnarRDD(
- @transient private val sc: SparkContext,
- metrics: Map[String, SQLMetric],
- numPartitioning: Int,
- inputByteBuf: broadcast.Broadcast[ColumnarHashedRelation])
- extends RDD[ColumnarBatch](sc, Nil) {
+ @transient private val sc: SparkContext,
+ metrics: Map[String, SQLMetric],
+ numPartitioning: Int,
+ inputByteBuf: broadcast.Broadcast[ColumnarHashedRelation])
+ extends RDD[ColumnarBatch](sc, Nil) {
override protected def getPartitions: Array[Partition] = {
(0 until numPartitioning).map { index => new BroadcastColumnarRDDPartition(index) }.toArray
diff --git a/jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala b/backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala
similarity index 67%
rename from jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala
rename to backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala
index cbf8c6bdc510..87595d8e9b4f 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala
+++ b/backends-velox/src/main/scala/io/glutenproject/execution/DataToArrowColumnarExec.scala
@@ -17,49 +17,17 @@
package io.glutenproject.execution
-import java.nio.ByteBuffer
-import java.util.concurrent.TimeUnit._
+import scala.collection.JavaConverters._
-import io.glutenproject.vectorized._
-import io.glutenproject.GlutenConfig
+import org.apache.spark.broadcast
-import org.apache.spark.{broadcast, TaskContext}
import org.apache.spark.rdd.RDD
-import org.apache.spark.util.{Utils, UserAddedJarUtils}
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{
- Attribute,
- Expression,
- SortOrder,
- UnsafeProjection
-}
-import org.apache.spark.sql.catalyst.expressions.codegen._
-import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeProjection}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
-
-import scala.collection.JavaConverters._
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.expressions.BoundReference
-import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference
-import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
-import scala.collection.mutable.ListBuffer
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.arrow.vector.types.pojo.Field
-import org.apache.arrow.vector.types.pojo.Schema
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.gandiva.evaluator._
-
-import org.apache.arrow.memory.ArrowBuf
-import io.netty.buffer.ByteBuf
-import com.google.common.collect.Lists;
-
-import io.glutenproject.expression._
-import io.glutenproject.vectorized.ExpressionEvaluator
-import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
+import org.apache.spark.sql.vectorized.ColumnarBatch
/**
* Performs a hash join of two child relations by first shuffling the data using the join keys.
diff --git a/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala b/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala
new file mode 100644
index 000000000000..af75964a947d
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/expression/ArrowConverterUtils.scala
@@ -0,0 +1,482 @@
+/*
+ * 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 io.glutenproject.expression
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, IOException, OutputStream}
+import java.nio.channels.Channels
+import scala.collection.JavaConverters._
+
+import com.google.common.collect.Lists
+import io.glutenproject.vectorized.ArrowWritableColumnVector
+import io.netty.buffer.{ByteBufAllocator, ByteBufOutputStream}
+import org.apache.arrow.flatbuf.MessageHeader
+import org.apache.arrow.gandiva.exceptions.GandivaException
+import org.apache.arrow.gandiva.expression.{ExpressionTree, TreeBuilder, TreeNode}
+import org.apache.arrow.gandiva.ipc.GandivaTypes
+import org.apache.arrow.gandiva.ipc.GandivaTypes.ExpressionList
+import org.apache.arrow.memory.BufferAllocator
+import org.apache.arrow.vector.ipc.{ReadChannel, WriteChannel}
+import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, IpcOption, MessageChannelReader, MessageResult, MessageSerializer}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}
+import org.apache.arrow.vector.ValueVector
+import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID
+import org.apache.arrow.vector.types.TimeUnit
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.DateTimeConstants
+import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkSchemaUtils, SparkVectorUtils}
+import org.apache.spark.sql.types.{ArrayType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, ShortType, StringType, StructField, StructType, TimestampType}
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
+
+object ArrowConverterUtils extends Logging {
+
+ def calcuateEstimatedSize(columnarBatch: ColumnarBatch): Long = {
+ SparkVectorUtils.estimateSize(columnarBatch)
+ }
+
+ def createArrowRecordBatch(columnarBatch: ColumnarBatch): ArrowRecordBatch = {
+ SparkVectorUtils.toArrowRecordBatch(columnarBatch)
+ }
+
+ def createArrowRecordBatch(numRowsInBatch: Int, cols: List[ValueVector]): ArrowRecordBatch = {
+ SparkVectorUtils.toArrowRecordBatch(numRowsInBatch, cols)
+ }
+
+ def convertToNetty(iter: Array[ColumnarBatch], out: OutputStream): Unit = {
+ val channel = new WriteChannel(Channels.newChannel(out))
+ var schema: Schema = null
+ val option = new IpcOption
+
+ iter.foreach { columnarBatch =>
+ val vectors = (0 until columnarBatch.numCols)
+ .map(i => columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector])
+ .toList
+ try {
+ if (schema == null) {
+ schema = new Schema(vectors.map(_.getValueVector().getField).asJava)
+ MessageSerializer.serialize(channel, schema, option)
+ }
+ val batch = ArrowConverterUtils
+ .createArrowRecordBatch(columnarBatch.numRows, vectors.map(_.getValueVector))
+ try {
+ MessageSerializer.serialize(channel, batch, option)
+ } finally {
+ batch.close()
+ }
+ } catch {
+ case e =>
+ System.err.println(s"conversion failed")
+ e.printStackTrace()
+ throw e
+ }
+ }
+ }
+
+ def convertToNetty(iter: Array[ColumnarBatch]): Array[Byte] = {
+ val innerBuf = ByteBufAllocator.DEFAULT.buffer()
+ val outStream = new ByteBufOutputStream(innerBuf)
+ convertToNetty(iter, outStream)
+ val bytes = new Array[Byte](innerBuf.readableBytes);
+ innerBuf.getBytes(innerBuf.readerIndex, bytes);
+ innerBuf.release()
+ outStream.close()
+ bytes
+ }
+
+ def convertFromNetty(
+ attributes: Seq[Attribute],
+ input: InputStream): Iterator[ColumnarBatch] = {
+ new Iterator[ColumnarBatch] {
+ val allocator = ArrowWritableColumnVector.getOffRecordAllocator
+ var messageReader =
+ new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
+ var schema: Schema = null
+ var result: MessageResult = null
+
+ override def hasNext: Boolean =
+ if (input.available > 0) {
+ return true
+ } else {
+ messageReader.close
+ return false
+ }
+ override def next(): ColumnarBatch = {
+ if (input.available == 0) {
+ if (attributes == null) {
+ return null
+ }
+ val resultStructType = StructType(
+ attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
+ return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
+ }
+ try {
+ if (schema == null) {
+ result = messageReader.readNext();
+ if (result == null) {
+ throw new IOException("Unexpected end of input. Missing schema.");
+ }
+ if (result.getMessage().headerType() != MessageHeader.Schema) {
+ throw new IOException(
+ "Expected schema but header was " + result.getMessage().headerType());
+ }
+ schema = MessageSerializer.deserializeSchema(result.getMessage());
+ }
+ result = messageReader.readNext();
+ if (result.getMessage().headerType() != MessageHeader.RecordBatch) {
+ throw new IOException(
+ "Expected recordbatch but header was " + result.getMessage().headerType());
+ }
+ var bodyBuffer = result.getBodyBuffer();
+
+ // For zero-length batches, need an empty buffer to deserialize the batch
+ if (bodyBuffer == null) {
+ bodyBuffer = allocator.getEmpty();
+ }
+
+ val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer);
+ val vectors = fromArrowRecordBatch(schema, batch, allocator)
+ val length = batch.getLength
+ batch.close
+ new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length)
+ } catch {
+ case e: Throwable =>
+ messageReader.close
+ throw e
+ }
+ }
+ }
+ }
+
+ def convertFromNetty(
+ attributes: Seq[Attribute],
+ data: Array[Array[Byte]],
+ columnIndices: Array[Int] = null): Iterator[ColumnarBatch] = {
+ if (data.size == 0) {
+ return new Iterator[ColumnarBatch] {
+ override def hasNext: Boolean = false
+ override def next(): ColumnarBatch = {
+ val resultStructType = if (columnIndices == null) {
+ StructType(
+ attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
+ } else {
+ StructType(
+ columnIndices
+ .map(i => attributes(i))
+ .map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
+ }
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
+ return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
+ }
+ }
+ }
+ new Iterator[ColumnarBatch] {
+ var array_id = 0
+ val allocator = ArrowWritableColumnVector.getOffRecordAllocator
+ var input = new ByteArrayInputStream(data(array_id))
+ var messageReader =
+ new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
+ var schema: Schema = null
+ var result: MessageResult = null
+
+ override def hasNext: Boolean =
+ if (array_id < (data.size - 1) || input.available > 0) {
+ return true
+ } else {
+ messageReader.close
+ return false
+ }
+ override def next(): ColumnarBatch = {
+ if (input.available == 0) {
+ messageReader.close
+ array_id += 1
+ input = new ByteArrayInputStream(data(array_id))
+ messageReader =
+ new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
+ }
+ if (input.available == 0) {
+ val resultStructType = StructType(
+ attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
+ val resultColumnVectors =
+ ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
+ return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
+ }
+ try {
+ if (schema == null) {
+ result = messageReader.readNext();
+
+ if (result == null) {
+ throw new IOException("Unexpected end of input. Missing schema.");
+ }
+
+ if (result.getMessage().headerType() != MessageHeader.Schema) {
+ throw new IOException(
+ "Expected schema but header was " + result.getMessage().headerType());
+ }
+
+ schema = MessageSerializer.deserializeSchema(result.getMessage());
+
+ }
+
+ result = messageReader.readNext();
+ if (result.getMessage().headerType() == MessageHeader.Schema) {
+ result = messageReader.readNext();
+ }
+
+ if (result.getMessage().headerType() != MessageHeader.RecordBatch) {
+ throw new IOException(
+ "Expected recordbatch but header was " + result.getMessage().headerType());
+ }
+ var bodyBuffer = result.getBodyBuffer();
+
+ // For zero-length batches, need an empty buffer to deserialize the batch
+ if (bodyBuffer == null) {
+ bodyBuffer = allocator.getEmpty();
+ }
+
+ val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer);
+ val vectors = fromArrowRecordBatch(schema, batch, allocator)
+ val length = batch.getLength
+ batch.close
+ if (columnIndices == null) {
+ new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length)
+ } else {
+ new ColumnarBatch(
+ columnIndices.map(i => vectors(i).asInstanceOf[ColumnVector]),
+ length)
+ }
+
+ } catch {
+ case e: Throwable =>
+ messageReader.close
+ throw e
+ }
+ }
+ }
+ }
+
+ def fromArrowRecordBatch(
+ recordBatchSchema: Schema,
+ recordBatch: ArrowRecordBatch,
+ allocator: BufferAllocator = null): Array[ArrowWritableColumnVector] = {
+ val numRows = recordBatch.getLength()
+ ArrowWritableColumnVector.loadColumns(numRows, recordBatchSchema, recordBatch, allocator)
+ }
+
+ def releaseArrowRecordBatch(recordBatch: ArrowRecordBatch): Unit = {
+ if (recordBatch != null) {
+ recordBatch.close()
+ }
+ }
+
+ def releaseArrowRecordBatchList(recordBatchList: Array[ArrowRecordBatch]): Unit = {
+ recordBatchList.foreach({ recordBatch =>
+ if (recordBatch != null)
+ releaseArrowRecordBatch(recordBatch)
+ })
+ }
+
+ def combineArrowRecordBatch(rb1: ArrowRecordBatch, rb2: ArrowRecordBatch): ArrowRecordBatch = {
+ val numRows = rb1.getLength()
+ val rb1_nodes = rb1.getNodes()
+ val rb2_nodes = rb2.getNodes()
+ val rb1_bufferlist = rb1.getBuffers()
+ val rb2_bufferlist = rb2.getBuffers()
+
+ val combined_nodes = rb1_nodes.addAll(rb2_nodes)
+ val combined_bufferlist = rb1_bufferlist.addAll(rb2_bufferlist)
+ new ArrowRecordBatch(numRows, rb1_nodes, rb1_bufferlist)
+ }
+
+ def toArrowSchema(attributes: Seq[Attribute]): Schema = {
+ val fields = attributes.map(attr => {
+ Field
+ .nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType))
+ })
+ new Schema(fields.toList.asJava)
+ }
+
+ def toArrowSchema(schema: StructType): Schema = {
+ val fields = schema
+ .map(field => {
+ Field.nullable(field.name, CodeGeneration.getResultType(field.dataType))
+ })
+ new Schema(fields.toList.asJava)
+ }
+
+
+ @throws[IOException]
+ def getSchemaBytesBuf(schema: Schema): Array[Byte] = {
+ val out: ByteArrayOutputStream = new ByteArrayOutputStream
+ MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema)
+ out.toByteArray
+ }
+
+ @throws[IOException]
+ def getSchemaFromBytesBuf(schema: Array[Byte]): Schema = {
+ val in: ByteArrayInputStream = new ByteArrayInputStream(schema)
+ MessageSerializer.deserializeSchema(new ReadChannel(Channels.newChannel(in)))
+ }
+
+ @throws[GandivaException]
+ def getExprListBytesBuf(exprs: List[ExpressionTree]): Array[Byte] = {
+ val builder: ExpressionList.Builder = GandivaTypes.ExpressionList.newBuilder
+ exprs.foreach { expr => builder.addExprs(expr.toProtobuf) }
+ builder.build.toByteArray
+ }
+
+ def checkIfTypeSupported(dt: DataType): Unit = dt match {
+ case d: BooleanType =>
+ case d: ByteType =>
+ case d: ShortType =>
+ case d: IntegerType =>
+ case d: LongType =>
+ case d: FloatType =>
+ case d: DoubleType =>
+ case d: StringType =>
+ case d: DateType =>
+ case d: DecimalType =>
+ case d: TimestampType =>
+ case _ =>
+ throw new UnsupportedOperationException(s"Unsupported data type: $dt")
+ }
+
+ def createArrowField(name: String, dt: DataType): Field = dt match {
+ case at: ArrayType =>
+ new Field(
+ name,
+ FieldType.nullable(ArrowType.List.INSTANCE),
+ Lists.newArrayList(createArrowField(s"${name}_${dt}", at.elementType)))
+ case mt: MapType =>
+ throw new UnsupportedOperationException(s"${dt} is not supported yet")
+ case st: StructType =>
+ throw new UnsupportedOperationException(s"${dt} is not supported yet")
+ case _ =>
+ Field.nullable(name, CodeGeneration.getResultType(dt))
+ }
+
+ def createArrowField(attr: Attribute): Field =
+ createArrowField(s"${attr.name}#${attr.exprId.id}", attr.dataType)
+
+ private def asTimestampType(inType: ArrowType): ArrowType.Timestamp = {
+ if (inType.getTypeID != ArrowTypeID.Timestamp) {
+ throw new IllegalArgumentException(s"Value type to convert must be timestamp")
+ }
+ inType.asInstanceOf[ArrowType.Timestamp]
+ }
+
+ def convertTimestampZone(inNode: TreeNode, inType: ArrowType,
+ toZone: String): (TreeNode, ArrowType) = {
+ throw new UnsupportedOperationException("not implemented") // fixme 20210602 hongze
+ val inTimestamp = asTimestampType(inType)
+ val fromZone = inTimestamp.getTimezone
+
+ val (outNode0: TreeNode, outTimestamp0: ArrowType.Timestamp) =
+ if (SparkSchemaUtils.timeZoneIDEquals(fromZone, toZone)) {
+ val outType = new ArrowType.Timestamp(inTimestamp.getUnit, toZone)
+ (inNode, outType)
+ } else {
+ // todo conversion
+ }
+ (outNode0, outTimestamp0)
+ }
+
+ def convertTimestampToMilli(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
+ val inTimestamp = asTimestampType(inType)
+ inTimestamp.getUnit match {
+ case TimeUnit.MILLISECOND => (inNode, inType)
+ case TimeUnit.MICROSECOND =>
+ // truncate from micro to milli
+ val outType = new ArrowType.Timestamp(TimeUnit.MILLISECOND,
+ inTimestamp.getTimezone)
+ (TreeBuilder.makeFunction(
+ "convertTimestampUnit",
+ Lists.newArrayList(inNode), outType), outType)
+ }
+ }
+
+ def convertTimestampToMicro(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
+ val inTimestamp = asTimestampType(inType)
+ inTimestamp.getUnit match {
+ case TimeUnit.MICROSECOND => (inNode, inType)
+ case TimeUnit.MILLISECOND =>
+ // truncate from micro to milli
+ val outType = new ArrowType.Timestamp(TimeUnit.MICROSECOND,
+ inTimestamp.getTimezone)
+ (TreeBuilder.makeFunction(
+ "convertTimestampUnit",
+ Lists.newArrayList(inNode), outType), outType)
+ }
+ }
+
+ def toInt32(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
+ val toType = ArrowUtils.toArrowType(IntegerType, null)
+ val toNode = TreeBuilder.makeFunction("castINT", Lists.newArrayList(inNode),
+ toType)
+ (toNode, toType)
+ }
+
+ // use this carefully
+ def toGandivaMicroUTCTimestamp(inNode: TreeNode, inType: ArrowType,
+ timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
+ val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get
+ val utcTimestampNodeOriginal = inNode
+ val inTimestampType = asTimestampType(inType)
+ val inTimestampTypeUTC = new ArrowType.Timestamp(inTimestampType.getUnit,
+ "UTC")
+ ArrowConverterUtils.convertTimestampToMicro(utcTimestampNodeOriginal,
+ inTimestampTypeUTC)
+ }
+
+ // use this carefully
+ def toGandivaTimestamp(inNode: TreeNode, inType: ArrowType,
+ timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
+ val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get
+
+ val utcTimestampNodeOriginal = inNode
+ val utcTimestampNodeMilli = ArrowConverterUtils.convertTimestampToMilli(
+ utcTimestampNodeOriginal,inType)._1
+ val utcTimestampNodeLong = TreeBuilder.makeFunction("castBIGINT",
+ Lists.newArrayList(utcTimestampNodeMilli), new ArrowType.Int(64,
+ true))
+ val diff = SparkSchemaUtils.getTimeZoneIDOffset(zoneId) *
+ DateTimeConstants.MILLIS_PER_SECOND
+
+ val localizedTimestampNodeLong = TreeBuilder.makeFunction("add",
+ Lists.newArrayList(utcTimestampNodeLong,
+ TreeBuilder.makeLiteral(java.lang.Long.valueOf(diff))),
+ new ArrowType.Int(64, true))
+ val localized = new ArrowType.Timestamp(TimeUnit.MILLISECOND, null)
+ val localizedTimestampNode = TreeBuilder.makeFunction("castTIMESTAMP",
+ Lists.newArrayList(localizedTimestampNodeLong), localized)
+ (localizedTimestampNode, localized)
+ }
+
+ def toSparkTimestamp(inNode: TreeNode, inType: ArrowType,
+ timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
+ throw new UnsupportedOperationException()
+ }
+
+ override def toString(): String = {
+ s"ArrowConverterUtils"
+ }
+}
diff --git a/jvm/src/main/scala/io/glutenproject/expression/CodeGeneration.scala b/backends-velox/src/main/scala/io/glutenproject/expression/CodeGeneration.scala
similarity index 100%
rename from jvm/src/main/scala/io/glutenproject/expression/CodeGeneration.scala
rename to backends-velox/src/main/scala/io/glutenproject/expression/CodeGeneration.scala
diff --git a/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala
new file mode 100644
index 000000000000..510bf0d42862
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxImplicitClass.scala
@@ -0,0 +1,43 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.utils
+
+import io.glutenproject.vectorized.ArrowWritableColumnVector
+import org.apache.arrow.vector.util.VectorBatchAppender
+
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+object VeloxImplicitClass {
+
+ implicit class ArrowColumnarBatchRetainer(val cb: ColumnarBatch) {
+ def retain(): Unit = {
+ (0 until cb.numCols).toList.foreach(i =>
+ cb.column(i).asInstanceOf[ArrowWritableColumnVector].retain())
+ }
+ }
+
+ def coalesce(targetBatch: ColumnarBatch, batchesToAppend: List[ColumnarBatch]): Unit = {
+ (0 until targetBatch.numCols).toList.foreach { i =>
+ val targetVector =
+ targetBatch.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector
+ val vectorsToAppend = batchesToAppend.map { cb =>
+ cb.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector
+ }
+ VectorBatchAppender.batchAppend(targetVector, vectorsToAppend: _*)
+ }
+ }
+}
diff --git a/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala
new file mode 100644
index 000000000000..3859561a14f9
--- /dev/null
+++ b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxTransformerUtil.scala
@@ -0,0 +1,71 @@
+/*
+ * 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 io.glutenproject.utils
+
+import scala.collection.JavaConverters._
+import com.google.common.collect.Lists
+
+import io.glutenproject.expression.CodeGeneration
+import org.apache.arrow.gandiva.expression.{TreeBuilder, TreeNode}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field}
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+object VeloxTransformerUtil {
+
+ def prepareRelationFunction(
+ keyAttributes: Seq[Attribute],
+ outputAttributes: Seq[Attribute]): TreeNode = {
+ val outputFieldList: List[Field] = outputAttributes.toList.map(attr => {
+ Field
+ .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}",
+ CodeGeneration.getResultType(attr.dataType))
+ })
+
+ val keyFieldList: List[Field] = keyAttributes.toList.map(attr => {
+ val field = Field
+ .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}",
+ CodeGeneration.getResultType(attr.dataType))
+ if (outputFieldList.indexOf(field) == -1) {
+ throw new UnsupportedOperationException(s"CachedRelation not found" +
+ s"${attr.name.toUpperCase()}#${attr.exprId.id} in ${outputAttributes}")
+ }
+ field
+ })
+
+ val key_args_node = TreeBuilder.makeFunction(
+ "key_field",
+ keyFieldList
+ .map(field => {
+ TreeBuilder.makeField(field)
+ })
+ .asJava,
+ new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ )
+
+ val cachedRelationFuncName = "CachedRelation"
+ val cached_relation_func = TreeBuilder.makeFunction(
+ cachedRelationFuncName,
+ Lists.newArrayList(key_args_node),
+ new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ )
+
+ TreeBuilder.makeFunction(
+ "standalone",
+ Lists.newArrayList(cached_relation_func),
+ new ArrowType.Int(32, true))
+ }
+}
diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala b/backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala
similarity index 91%
rename from jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala
rename to backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala
index 92575c6e708f..ab0dd7a37643 100644
--- a/jvm/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala
+++ b/backends-velox/src/main/scala/io/glutenproject/vectorized/ArrowColumnarBatchSerializer.scala
@@ -26,29 +26,22 @@ import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag
import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.ConverterUtils
+import io.glutenproject.expression.ArrowConverterUtils
import org.apache.arrow.dataset.jni.UnsafeRecordBatchSerializer
-import org.apache.arrow.memory.ArrowBuf
-import org.apache.arrow.memory.BufferAllocator
+import org.apache.arrow.memory.{ArrowBuf, BufferAllocator}
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
import org.apache.arrow.vector.ipc.ArrowStreamReader
-import org.apache.arrow.vector.VectorLoader
-import org.apache.arrow.vector.VectorSchemaRoot
import org.apache.arrow.vector.types.pojo.Schema
-
import org.apache.spark.SparkEnv
+
import org.apache.spark.internal.Logging
-import org.apache.spark.serializer.DeserializationStream
-import org.apache.spark.serializer.SerializationStream
-import org.apache.spark.serializer.Serializer
-import org.apache.spark.serializer.SerializerInstance
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkVectorUtils
+import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance}
+import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkMemoryUtils, SparkVectorUtils}
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.ColumnVector
-import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
class ArrowColumnarBatchSerializer(
schema: StructType, readBatchNumRows: SQLMetric, numOutputRows: SQLMetric)
@@ -198,7 +191,7 @@ private class ArrowColumnarBatchSerializerInstance(
private def decompressVectors(): Unit = {
if (jniWrapper == null) {
jniWrapper = new ShuffleDecompressionJniWrapper
- schemaHolderId = jniWrapper.make(ConverterUtils.getSchemaBytesBuf(root.getSchema))
+ schemaHolderId = jniWrapper.make(ArrowConverterUtils.getSchemaBytesBuf(root.getSchema))
}
if (vectorLoader == null) {
vectorLoader = new VectorLoader(root)
diff --git a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala b/backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala
similarity index 92%
rename from jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala
rename to backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala
index 23575c978d90..4068a8fb901e 100644
--- a/jvm/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala
+++ b/backends-velox/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala
@@ -17,17 +17,18 @@
package io.glutenproject.vectorized
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
import org.apache.spark.TaskContext
+
+import org.apache.spark.internal.Logging
import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
+import org.apache.spark.sql.vectorized.ColumnarBatch
/**
* An Iterator that insures that the batches [[ColumnarBatch]]s it iterates over are all closed
* properly.
*/
class CloseableColumnBatchIterator(itr: Iterator[ColumnarBatch])
- extends Iterator[ColumnarBatch]
+ extends Iterator[ColumnarBatch]
with Logging {
var cb: ColumnarBatch = null
@@ -40,8 +41,8 @@ class CloseableColumnBatchIterator(itr: Iterator[ColumnarBatch])
}
SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit]((tc: TaskContext) => {
- closeCurrentBatch()
- })
+ closeCurrentBatch()
+ })
override def hasNext: Boolean = {
itr.hasNext
diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala
similarity index 71%
rename from jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala
rename to backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala
index 4f214fd78fa2..7c7f0fe65ecc 100644
--- a/jvm/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/VeloxColumnarShuffleWriter.scala
@@ -24,7 +24,7 @@ import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import com.google.common.annotations.VisibleForTesting
import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.ConverterUtils
+import io.glutenproject.expression.ArrowConverterUtils
import io.glutenproject.spark.sql.execution.datasources.v2.arrow.Spiller
import io.glutenproject.vectorized._
import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID
@@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.Utils
-class ColumnarShuffleWriter[K, V](
+class VeloxColumnarShuffleWriter[K, V](
shuffleBlockResolver: IndexShuffleBlockResolver,
handle: BaseShuffleHandle[K, V, V],
mapId: Long,
@@ -80,7 +80,7 @@ class ColumnarShuffleWriter[K, V](
private val writeSchema = GlutenConfig.getConf.columnarShuffleWriteSchema
- private val jniWrapper = ColumnarFactory.createShuffleSplitterJniWrapper();
+ private val jniWrapper = new ShuffleSplitterJniWrapper
private var nativeSplitter: Long = 0
@@ -139,7 +139,7 @@ class ColumnarShuffleWriter[K, V](
val startTimeForPrepare = System.nanoTime()
val bufAddrs = new ListBuffer[Long]()
val bufSizes = new ListBuffer[Long]()
- val recordBatch = ConverterUtils.createArrowRecordBatch(cb)
+ val recordBatch = ArrowConverterUtils.createArrowRecordBatch(cb)
recordBatch.getBuffers().asScala.foreach { buffer => bufAddrs += buffer.memoryAddress() }
recordBatch.getBuffersLayout().asScala.foreach { bufLayout =>
bufSizes += bufLayout.getSize()
@@ -149,8 +149,8 @@ class ColumnarShuffleWriter[K, V](
val startTime = System.nanoTime()
val existingIntType: Boolean = if (firstRecordBatch) {
// Check whether the recordbatch contain the Int data type.
- val arrowSchema = ConverterUtils.getSchemaFromBytesBuf(dep.nativePartitioning.getSchema)
- import scala.collection.JavaConverters._
+ val arrowSchema = ArrowConverterUtils.getSchemaFromBytesBuf(
+ dep.nativePartitioning.getSchema)
arrowSchema.getFields.asScala.find(_.getType.getTypeID == ArrowTypeID.Int).nonEmpty
} else false
@@ -184,7 +184,7 @@ class ColumnarShuffleWriter[K, V](
dep.splitTime.add(System.nanoTime() - startTime)
dep.numInputRows.add(cb.numRows)
writeMetrics.incRecordsWritten(1)
- ConverterUtils.releaseArrowRecordBatch(recordBatch)
+ ArrowConverterUtils.releaseArrowRecordBatch(recordBatch)
}
}
@@ -222,95 +222,15 @@ class ColumnarShuffleWriter[K, V](
mapStatus = MapStatus(blockManager.shuffleServerId, unionPartitionLengths.toArray, mapId)
}
- def internalCHWrite(records: Iterator[Product2[K, V]]): Unit = {
- val splitterJniWrapper : CHShuffleSplitterJniWrapper =
- jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper]
- if (!records.hasNext) {
- partitionLengths = new Array[Long](dep.partitioner.numPartitions)
- shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, null)
- mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId)
- return
- }
- val dataTmp = Utils.tempFileWith(shuffleBlockResolver.getDataFile(dep.shuffleId, mapId))
- if (nativeSplitter == 0) {
- nativeSplitter = splitterJniWrapper.make(
- dep.nativePartitioning,
- mapId,
- nativeBufferSize,
- defaultCompressionCodec,
- dataTmp.getAbsolutePath,
- localDirs)
- }
- while (records.hasNext) {
- val cb = records.next()._2.asInstanceOf[ColumnarBatch]
- if (cb.numRows == 0 || cb.numCols == 0) {
- logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols")
- } else {
- val startTimeForPrepare = System.nanoTime()
-
- val startTime = System.nanoTime()
- firstRecordBatch = false
- dep.prepareTime.add(System.nanoTime() - startTimeForPrepare)
- val col = cb.column(0).asInstanceOf[CHColumnVector]
- val block = col.getBlockAddress
- splitterJniWrapper
- .split(nativeSplitter, cb.numRows, block)
- dep.splitTime.add(System.nanoTime() - startTime)
- dep.numInputRows.add(cb.numRows)
- writeMetrics.incRecordsWritten(1)
- }
- }
- val startTime = System.nanoTime()
- splitResult = splitterJniWrapper.stop(nativeSplitter)
-
- dep.splitTime.add(System.nanoTime() - startTime - splitResult.getTotalSpillTime -
- splitResult.getTotalWriteTime - splitResult.getTotalComputePidTime -
- splitResult.getTotalCompressTime)
- dep.spillTime.add(splitResult.getTotalSpillTime)
- dep.compressTime.add(splitResult.getTotalCompressTime)
- dep.computePidTime.add(splitResult.getTotalComputePidTime)
- dep.bytesSpilled.add(splitResult.getTotalBytesSpilled)
- writeMetrics.incBytesWritten(splitResult.getTotalBytesWritten)
- writeMetrics.incWriteTime(splitResult.getTotalWriteTime + splitResult.getTotalSpillTime)
-
- partitionLengths = splitResult.getPartitionLengths
- rawPartitionLengths = splitResult.getRawPartitionLengths
- try {
- shuffleBlockResolver.writeIndexFileAndCommit(
- dep.shuffleId,
- mapId,
- partitionLengths,
- dataTmp)
- } finally {
- if (dataTmp.exists() && !dataTmp.delete()) {
- logError(s"Error while deleting temp file ${dataTmp.getAbsolutePath}")
- }
- }
-
- // fixme workaround: to store uncompressed sizes on the rhs of (maybe) compressed sizes
- val unionPartitionLengths = ArrayBuffer[Long]()
- unionPartitionLengths ++= partitionLengths
- unionPartitionLengths ++= rawPartitionLengths
- mapStatus = MapStatus(blockManager.shuffleServerId, unionPartitionLengths.toArray, mapId)
- }
-
@throws[IOException]
override def write(records: Iterator[Product2[K, V]]): Unit = {
- if (GlutenConfig.getConf.isClickHouseBackend) {
- internalCHWrite(records)
- } else {
- internalWrite(records)
- }
+ internalWrite(records)
}
def closeSplitter(): Unit = {
jniWrapper.asInstanceOf[ShuffleSplitterJniWrapper].close(nativeSplitter)
}
- def closeCHSplitter(): Unit = {
- jniWrapper.asInstanceOf[CHShuffleSplitterJniWrapper].close(nativeSplitter)
- }
-
override def stop(success: Boolean): Option[MapStatus] = {
try {
if (stopping) {
@@ -324,11 +244,7 @@ class ColumnarShuffleWriter[K, V](
}
} finally {
if (nativeSplitter != 0) {
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- closeSplitter()
- } else {
- closeCHSplitter()
- }
+ closeSplitter()
nativeSplitter = 0
}
}
diff --git a/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala
new file mode 100644
index 000000000000..bee1fdfce075
--- /dev/null
+++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/utils/VeloxShuffleUtil.scala
@@ -0,0 +1,31 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package org.apache.spark.shuffle.utils
+
+import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper, VeloxColumnarShuffleWriter}
+
+object VeloxShuffleUtil {
+
+ def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V]
+ ): GlutenShuffleWriterWrapper[K, V] = {
+ GlutenShuffleWriterWrapper(new VeloxColumnarShuffleWriter[K, V](
+ parameters.shuffleBlockResolver,
+ parameters.columnarShuffleHandle,
+ parameters.mapId,
+ parameters.metrics))
+ }
+}
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala
similarity index 91%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala
index 25a614b71724..9e65de295889 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarHashedRelation.scala
@@ -19,14 +19,15 @@ package org.apache.spark.sql.execution
import java.io._
-import com.esotericsoftware.kryo.io.{Input, Output}
import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
-import io.glutenproject.expression.ConverterUtils
+import com.esotericsoftware.kryo.io.{Input, Output}
+import io.glutenproject.expression.ArrowConverterUtils
import io.glutenproject.vectorized.{ArrowWritableColumnVector, SerializableObject}
+import sun.misc.Cleaner
+
import org.apache.spark.sql.execution.ColumnarHashedRelation.Deallocator
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.KnownSizeEstimation
-import sun.misc.Cleaner
class ColumnarHashedRelation(
var hashRelationObj: SerializableObject,
@@ -60,13 +61,13 @@ class ColumnarHashedRelation(
override def writeExternal(out: ObjectOutput): Unit = {
out.writeObject(hashRelationObj)
- val rawArrowData = ConverterUtils.convertToNetty(arrowColumnarBatch)
+ val rawArrowData = ArrowConverterUtils.convertToNetty(arrowColumnarBatch)
out.writeObject(rawArrowData)
}
override def write(kryo: Kryo, out: Output): Unit = {
kryo.writeObject(out, hashRelationObj)
- val rawArrowData = ConverterUtils.convertToNetty(arrowColumnarBatch)
+ val rawArrowData = ArrowConverterUtils.convertToNetty(arrowColumnarBatch)
kryo.writeObject(out, rawArrowData)
}
@@ -75,7 +76,7 @@ class ColumnarHashedRelation(
val rawArrowData = in.readObject().asInstanceOf[Array[Byte]]
arrowColumnarBatchSize = rawArrowData.length
arrowColumnarBatch =
- ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
+ ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
createCleaner(hashRelationObj, arrowColumnarBatch)
// retain all cols
/*arrowColumnarBatch.foreach(cb => {
@@ -90,7 +91,7 @@ class ColumnarHashedRelation(
val rawArrowData = kryo.readObject(in, classOf[Array[Byte]]).asInstanceOf[Array[Byte]]
arrowColumnarBatchSize = rawArrowData.length
arrowColumnarBatch =
- ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
+ ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
createCleaner(hashRelationObj, arrowColumnarBatch)
// retain all cols
/*arrowColumnarBatch.foreach(cb => {
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala
similarity index 85%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala
index e28124c658d7..2ce1f17435f3 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarInMemoryRelation.scala
@@ -18,39 +18,26 @@
package org.apache.spark.sql.execution
import java.io._
-import org.apache.commons.lang3.StringUtils
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import com.esotericsoftware.kryo.io.{Input, Output}
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.expression._
import io.glutenproject.vectorized.ArrowWritableColumnVector
-import io.glutenproject.vectorized.CloseableColumnBatchIterator
-import org.apache.arrow.memory.ArrowBuf
-import org.apache.spark.TaskContext
-import org.apache.spark.network.util.JavaUtils
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
-import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
-import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.columnar.{
- CachedBatch,
- CachedBatchSerializer,
- SimpleMetricsCachedBatch,
- SimpleMetricsCachedBatchSerializer
-}
+import org.apache.spark.sql.columnar.{CachedBatch, SimpleMetricsCachedBatch}
import org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.vectorized.{WritableColumnVector}
-import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.{LongAccumulator, Utils, KnownSizeEstimation}
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-import com.esotericsoftware.kryo.io.{Input, Output}
-import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import org.apache.spark.util.KnownSizeEstimation
/**
* The default implementation of CachedBatch.
@@ -77,20 +64,20 @@ case class ArrowCachedBatch(
override def estimatedSize: Long = {
var size: Long = 0
buffer.foreach(batch => {
- size += ConverterUtils.calcuateEstimatedSize(batch)
+ size += ArrowConverterUtils.calcuateEstimatedSize(batch)
})
size
}
override def sizeInBytes: Long = estimatedSize
override def writeExternal(out: ObjectOutput): Unit = {
out.writeObject(numRows)
- val rawArrowData = ConverterUtils.convertToNetty(buffer)
+ val rawArrowData = ArrowConverterUtils.convertToNetty(buffer)
out.writeObject(rawArrowData)
buffer.foreach(_.close)
}
override def write(kryo: Kryo, out: Output): Unit = {
kryo.writeObject(out, numRows)
- val rawArrowData = ConverterUtils.convertToNetty(buffer)
+ val rawArrowData = ArrowConverterUtils.convertToNetty(buffer)
kryo.writeObject(out, rawArrowData)
System.out.println("ArrowCachedBatch close when write to disk")
buffer.foreach(_.close)
@@ -98,12 +85,12 @@ case class ArrowCachedBatch(
override def readExternal(in: ObjectInput): Unit = {
numRows = in.readObject().asInstanceOf[Integer]
val rawArrowData = in.readObject().asInstanceOf[Array[Byte]]
- buffer = ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
+ buffer = ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
}
override def read(kryo: Kryo, in: Input): Unit = {
numRows = kryo.readObject(in, classOf[Integer]).asInstanceOf[Integer]
val rawArrowData = kryo.readObject(in, classOf[Array[Byte]]).asInstanceOf[Array[Byte]]
- buffer = ConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
+ buffer = ArrowConverterUtils.convertFromNetty(null, new ByteArrayInputStream(rawArrowData)).toArray
}
}
@@ -211,7 +198,7 @@ class ArrowColumnarCachedBatchSerializer extends DefaultCachedBatchSerializer {
}
def hasNext: Boolean = iter.hasNext
}
- new CloseableColumnBatchIterator(res)
+ BackendsApiManager.getIteratorApiInstance.genCloseableColumnBatchIterator(res)
}
input.mapPartitions(createAndDecompressColumn)
} else {
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala
similarity index 95%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala
index 2bec6536b987..0d57ed8f53cb 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkMemoryUtils.scala
@@ -17,25 +17,18 @@
package org.apache.spark.sql.execution.datasources.v2.arrow
-import io.glutenproject.GlutenConfig
-
import java.io.PrintWriter
-import java.util
import java.util.UUID
import scala.collection.JavaConverters._
-import io.glutenproject.spark.sql.execution.datasources.v2.arrow._
import com.sun.xml.internal.messaging.saaj.util.ByteOutputStream
+import io.glutenproject.spark.sql.execution.datasources.v2.arrow._
+import java.util
import org.apache.arrow.dataset.jni.NativeMemoryPool
-import org.apache.arrow.memory.AllocationListener
-import org.apache.arrow.memory.BufferAllocator
-import org.apache.arrow.memory.MemoryChunkCleaner
-import org.apache.arrow.memory.MemoryChunkManager
-import org.apache.arrow.memory.RootAllocator
-
-import org.apache.spark.SparkEnv
-import org.apache.spark.TaskContext
+import org.apache.arrow.memory.{AllocationListener, BufferAllocator, MemoryChunkCleaner, MemoryChunkManager, RootAllocator}
+import org.apache.spark.{SparkEnv, TaskContext}
+
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.memory.TaskMemoryManager
@@ -111,9 +104,7 @@ object SparkMemoryUtils extends Logging {
.newChildAllocator("CHILD-ALLOC-BUFFER-IMPORT", allocListenerForBufferImport, 0L,
Long.MaxValue)
- val defaultMemoryPool: NativeMemoryPoolWrapper = if (GlutenConfig.getConf.isClickHouseBackend) {
- null
- } else {
+ val defaultMemoryPool: NativeMemoryPoolWrapper = {
val rl = new SparkManagedReservationListener(
new NativeSQLMemoryConsumer(getTaskMemoryManager(), Spiller.NO_OP),
sharedMetrics)
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala
similarity index 100%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkSchemaUtils.scala
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala
similarity index 100%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/v2/arrow/SparkVectorUtils.scala
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala
similarity index 86%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala
index c26a033385d9..a4d6e633b6e9 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExecTransformer.scala
@@ -17,29 +17,18 @@
package org.apache.spark.sql.execution.python
-import java.io._
-import java.net._
-import java.util.concurrent.atomic.AtomicBoolean
-
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.execution.{TransformContext, TransformSupport}
-import io.glutenproject.expression._
import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized._
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-import org.apache.arrow.vector.{ValueVector, VectorLoader, VectorSchemaRoot}
-import org.apache.spark.{ContextAwareIterator, SparkEnv}
import org.apache.spark.TaskContext
+
import org.apache.spark.api.python.ChainedPythonFunctions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
-import org.apache.spark.sql.execution.python.EvalPythonExec
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnVector, ColumnarBatch}
-import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.ArrowUtils
import org.apache.spark.rdd.RDD
@@ -77,7 +66,8 @@ case class ArrowEvalPythonExecTransformer(udfs: Seq[PythonUDF], resultAttrs: Seq
val outputTypes = output.drop(child.output.length).map(_.dataType)
// Use Coalecse to improve performance in future
- val batchIter = new CloseableColumnBatchIterator(iter)
+ val batchIter =
+ BackendsApiManager.getIteratorApiInstance.genCloseableColumnBatchIterator(iter)
val columnarBatchIter = new ColumnarArrowPythonRunner(
funcs,
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala
similarity index 94%
rename from jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala
index a91371f07e22..e6e48c343a9a 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/python/ColumnarArrowPythonRunner.scala
@@ -23,19 +23,17 @@ import java.util.concurrent.atomic.AtomicBoolean
import io.glutenproject.expression._
import io.glutenproject.vectorized._
-
-import org.apache.arrow.vector.{ValueVector, VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter}
-import org.apache.spark.SparkEnv
-import org.apache.spark.TaskContext
+import org.apache.spark.{SparkEnv, TaskContext}
+
import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions, PythonRDD, SpecialLengths}
//import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution.python.PythonUDFRunner
import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
import org.apache.spark.util.Utils
/**
@@ -159,10 +157,10 @@ class ColumnarArrowPythonRunner(
while (inputIterator.hasNext) {
val nextBatch = inputIterator.next()
numRows += nextBatch.numRows
- val next_rb = ConverterUtils.createArrowRecordBatch(nextBatch)
+ val next_rb = ArrowConverterUtils.createArrowRecordBatch(nextBatch)
loader.load(next_rb)
writer.writeBatch()
- ConverterUtils.releaseArrowRecordBatch(next_rb)
+ ArrowConverterUtils.releaseArrowRecordBatch(next_rb)
}
// end writes footer to the output stream and doesn't clean any resources.
// It could throw exception if the output stream is closed, so it should be
diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala
new file mode 100644
index 000000000000..81e8e030c94a
--- /dev/null
+++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/VeloxExecUtil.scala
@@ -0,0 +1,241 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package org.apache.spark.sql.execution.utils
+
+import scala.collection.JavaConverters._
+
+import io.glutenproject.expression.{ArrowConverterUtils, ExpressionConverter, ExpressionTransformer}
+import io.glutenproject.substrait.expression.ExpressionNode
+import io.glutenproject.substrait.rel.RelBuilder
+import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativePartitioning}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+import org.apache.spark.{Partitioner, RangePartitioner, ShuffleDependency}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.shuffle.ColumnarShuffleDependency
+import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
+import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection}
+import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.PartitionIdPassthrough
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{IntegerType, StructType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.util.MutablePair
+
+object VeloxExecUtil {
+
+ def genShuffleDependency(rdd: RDD[ColumnarBatch],
+ outputAttributes: Seq[Attribute],
+ newPartitioning: Partitioning,
+ serializer: Serializer,
+ writeMetrics: Map[String, SQLMetric],
+ dataSize: SQLMetric,
+ bytesSpilled: SQLMetric,
+ numInputRows: SQLMetric,
+ computePidTime: SQLMetric,
+ splitTime: SQLMetric,
+ spillTime: SQLMetric,
+ compressTime: SQLMetric,
+ prepareTime: SQLMetric
+ ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = {
+ val arrowFields = outputAttributes.map(attr =>
+ ArrowConverterUtils.createArrowField(attr)
+ )
+ def serializeSchema(fields: Seq[Field]): Array[Byte] = {
+ val schema = new Schema(fields.asJava)
+ ArrowConverterUtils.getSchemaBytesBuf(schema)
+ }
+
+ // only used for fallback range partitioning
+ val rangePartitioner: Option[Partitioner] = newPartitioning match {
+ case RangePartitioning(sortingExpressions, numPartitions) =>
+ // Extract only fields used for sorting to avoid collecting large fields that does not
+ // affect sorting result when deciding partition bounds in RangePartitioner
+ val rddForSampling = rdd.mapPartitionsInternal { iter =>
+ // Internally, RangePartitioner runs a job on the RDD that samples keys to compute
+ // partition bounds. To get accurate samples, we need to copy the mutable keys.
+ iter.flatMap(batch => {
+ val rows = batch.rowIterator.asScala
+ val projection =
+ UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes)
+ val mutablePair = new MutablePair[InternalRow, Null]()
+ rows.map(row => mutablePair.update(projection(row).copy(), null))
+ })
+ }
+ // Construct ordering on extracted sort key.
+ val orderingAttributes = sortingExpressions.zipWithIndex.map {
+ case (ord, i) =>
+ ord.copy(child = BoundReference(i, ord.dataType, ord.nullable))
+ }
+ implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes)
+ val part = new RangePartitioner(
+ numPartitions,
+ rddForSampling,
+ ascending = true,
+ samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
+ Some(part)
+ case _ => None
+ }
+
+ // only used for fallback range partitioning
+ def computeAndAddPartitionId(cbIter: Iterator[ColumnarBatch],
+ partitionKeyExtractor: InternalRow => Any
+ ): CloseablePairedColumnarBatchIterator = {
+ CloseablePairedColumnarBatchIterator {
+ cbIter
+ .filter(cb => cb.numRows != 0 && cb.numCols != 0)
+ .map { cb =>
+ val startTime = System.nanoTime()
+ val pidVec = ArrowWritableColumnVector
+ .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType))
+ .head
+ (0 until cb.numRows).foreach { i =>
+ val row = cb.getRow(i)
+ val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row))
+ pidVec.putInt(i, pid)
+ }
+
+ val newColumns = (pidVec +: (0 until cb.numCols).map(cb.column)).toArray
+ newColumns.foreach(
+ _.asInstanceOf[ArrowWritableColumnVector].getValueVector.setValueCount(cb.numRows))
+ computePidTime.add(System.nanoTime() - startTime)
+ (0, new ColumnarBatch(newColumns, cb.numRows))
+ }
+ }
+ }
+
+ val nativePartitioning: NativePartitioning = newPartitioning match {
+ case SinglePartition => new NativePartitioning("single", 1, serializeSchema(arrowFields))
+ case RoundRobinPartitioning(n) =>
+ new NativePartitioning("rr", n, serializeSchema(arrowFields))
+ case HashPartitioning(exprs, n) =>
+ // Function map is not expected to be used.
+ val functionMap = new java.util.HashMap[String, java.lang.Long]()
+ val exprNodeList = new java.util.ArrayList[ExpressionNode]()
+ exprs.foreach(expr => {
+ exprNodeList.add(ExpressionConverter
+ .replaceWithExpressionTransformer(expr, outputAttributes)
+ .asInstanceOf[ExpressionTransformer]
+ .doTransform(functionMap))
+ })
+ val projectRel = RelBuilder.makeProjectRel(null, exprNodeList)
+ new NativePartitioning(
+ "hash",
+ n,
+ serializeSchema(arrowFields),
+ projectRel.toProtobuf().toByteArray)
+ // range partitioning fall back to row-based partition id computation
+ case RangePartitioning(orders, n) =>
+ val pidField = Field.nullable("pid", new ArrowType.Int(32, true))
+ new NativePartitioning("range", n, serializeSchema(pidField +: arrowFields))
+ }
+
+ val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] &&
+ newPartitioning.numPartitions > 1
+
+ // RDD passed to ShuffleDependency should be the form of key-value pairs.
+ // ColumnarShuffleWriter will compute ids from ColumnarBatch on native side other than read the "key" part.
+ // Thus in Columnar Shuffle we never use the "key" part.
+ val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition
+
+ val rddWithDummyKey: RDD[Product2[Int, ColumnarBatch]] = newPartitioning match {
+ case RangePartitioning(sortingExpressions, _) =>
+ rdd.mapPartitionsWithIndexInternal((_, cbIter) => {
+ val partitionKeyExtractor: InternalRow => Any = {
+ val projection =
+ UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes)
+ row => projection(row)
+ }
+ val newIter = computeAndAddPartitionId(cbIter, partitionKeyExtractor)
+
+ SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ =>
+ newIter.closeAppendedVector()
+ }
+
+ newIter
+ }, isOrderSensitive = isOrderSensitive)
+ case _ =>
+ rdd.mapPartitionsWithIndexInternal(
+ (_, cbIter) =>
+ cbIter.map { cb =>
+ (0 until cb.numCols).foreach(
+ cb.column(_)
+ .asInstanceOf[ArrowWritableColumnVector]
+ .getValueVector
+ .setValueCount(cb.numRows))
+ (0, cb)
+ },
+ isOrderSensitive = isOrderSensitive)
+ }
+
+ val dependency =
+ new ColumnarShuffleDependency[Int, ColumnarBatch, ColumnarBatch](
+ rddWithDummyKey,
+ new PartitionIdPassthrough(newPartitioning.numPartitions),
+ serializer,
+ shuffleWriterProcessor =
+ ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics),
+ nativePartitioning = nativePartitioning,
+ dataSize = dataSize,
+ bytesSpilled = bytesSpilled,
+ numInputRows = numInputRows,
+ computePidTime = computePidTime,
+ splitTime = splitTime,
+ spillTime = spillTime,
+ compressTime = compressTime,
+ prepareTime = prepareTime)
+
+ dependency
+ }
+}
+
+
+case class CloseablePairedColumnarBatchIterator(iter: Iterator[(Int, ColumnarBatch)])
+ extends Iterator[(Int, ColumnarBatch)]
+ with Logging {
+
+ private var cur: (Int, ColumnarBatch) = _
+
+ def closeAppendedVector(): Unit = {
+ if (cur != null) {
+ logDebug("Close appended partition id vector")
+ cur match {
+ case (_, cb: ColumnarBatch) =>
+ cb.column(0).asInstanceOf[ArrowWritableColumnVector].close()
+ }
+ cur = null
+ }
+ }
+
+ override def hasNext: Boolean = {
+ iter.hasNext
+ }
+
+ override def next(): (Int, ColumnarBatch) = {
+ closeAppendedVector()
+ if (iter.hasNext) {
+ cur = iter.next()
+ cur
+ } else Iterator.empty.next()
+ }
+}
\ No newline at end of file
diff --git a/jvm/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
similarity index 100%
rename from jvm/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
rename to backends-velox/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml
new file mode 100644
index 000000000000..804a178a5fe2
--- /dev/null
+++ b/dev/checkstyle-suppressions.xml
@@ -0,0 +1,51 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml
new file mode 100644
index 000000000000..39b7b158cefb
--- /dev/null
+++ b/dev/checkstyle.xml
@@ -0,0 +1,181 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/docs/ArrowBackend.md b/docs/ArrowBackend.md
index 1068b9f0edc1..9b8b1d16bf6b 100644
--- a/docs/ArrowBackend.md
+++ b/docs/ArrowBackend.md
@@ -99,7 +99,7 @@ $ popd
```bash
$ git clone https://github.com/oap-project/gluten.git
$ pushd gluten
-$ mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_protobuf=ON -Dbuild_gazelle_cpp=ON -Darrow_root=${ARROW_HOME}
+$ mvn clean package -Pbackends-gazelle -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_protobuf=ON -Dbuild_gazelle_cpp=ON -Darrow_root=${ARROW_HOME}
$ popd
```
diff --git a/docs/ClickHouse.md b/docs/ClickHouse.md
index c91701856701..66c9e79d0141 100644
--- a/docs/ClickHouse.md
+++ b/docs/ClickHouse.md
@@ -114,7 +114,7 @@ The prerequisites are the same as the one above mentioned. Compile Gluten with C
git clone https://github.com/oap-project/gluten.git
cd gluten/
export MAVEN_OPTS="-Xmx8g -XX:ReservedCodeCacheSize=2g"
- mvn clean install -Phadoop-2.7.4 -Pspark-3.1.1 -Dhadoop.version=2.8.5 -DskipTests -Dbuild_cpp=OFF -Dcpp_tests=OFF -Dbuild_arrow=ON -Dbuild_protobuf=ON -Dbuild_jemalloc=ON -Dcheckstyle.skip
+ mvn clean install -Pbackends-clickhouse -Phadoop-2.7.4 -Pspark-3.1.1 -Dhadoop.version=2.8.5 -DskipTests -Dbuild_cpp=OFF -Dcpp_tests=OFF -Dbuild_arrow=ON -Dbuild_protobuf=ON -Dbuild_jemalloc=ON -Dcheckstyle.skip
ls -al jvm/target/gluten-jvm-XXXXX-jar-with-dependencies.jar
```
diff --git a/docs/GlutenUsage.md b/docs/GlutenUsage.md
index 2db1aef10e6e..65d6083035a4 100644
--- a/docs/GlutenUsage.md
+++ b/docs/GlutenUsage.md
@@ -25,7 +25,7 @@ If you wish to enable Velox backend and you have an existing compiled Velox, ple
The full compiling command would be like:
```shell script
-mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dbuild_velox_from_source=ON
+mvn clean package -Pbackends-velox -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dbuild_velox_from_source=ON
```
If Arrow has once been installed successfully on your env, and there is no change to Arrow, you can
diff --git a/docs/Velox.md b/docs/Velox.md
index a3b16a40ae74..643e39e4f081 100644
--- a/docs/Velox.md
+++ b/docs/Velox.md
@@ -48,7 +48,7 @@ After Velox being successfully compiled, please refer to [GlutenUsage](GlutenUsa
use below command to compile Gluten with Velox backend.
```shell script
-mvn clean package -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dvelox_home=${VELOX_HOME}
+mvn clean package -Pbackends-velox -P full-scala-compiler -DskipTests -Dcheckstyle.skip -Dbuild_cpp=ON -Dbuild_velox=ON -Dvelox_home=${VELOX_HOME}
```
### An example for offloading Spark's computing to Velox with Gluten
diff --git a/jvm/pom.xml b/jvm/pom.xml
index 0b3790807d0b..f9ec97828a4e 100644
--- a/jvm/pom.xml
+++ b/jvm/pom.xml
@@ -24,7 +24,7 @@
gluten-jvm
1.0.0-snapshot
jar
- Gluten
+ Gluten-JVM
../cpp/
@@ -77,97 +77,6 @@
spark-hive-thriftserver_${scala.binary.version}
provided
-
- io.delta
- delta-core_${scala.binary.version}
- provided
-
-
- org.apache.arrow
- ${arrow-memory.artifact}
- ${arrow.version}
- runtime
-
-
- org.apache.arrow
- arrow-memory-core
- ${arrow.version}
-
-
- io.netty
- netty-common
-
-
- io.netty
- netty-buffer
-
-
-
-
-
- org.apache.arrow
- arrow-vector
- ${arrow.version}
-
-
- io.netty
- netty-common
-
-
- io.netty
- netty-buffer
-
-
-
-
-
- org.apache.arrow.gandiva
- arrow-gandiva
- ${arrow.version}
-
-
- io.netty
- netty-common
-
-
- io.netty
- netty-buffer
-
-
- protobuf-java
- com.google.protobuf
-
-
-
-
-
- org.apache.arrow
- arrow-dataset
- ${arrow.version}
-
-
- io.netty
- netty-common
-
-
- io.netty
- netty-buffer
-
-
- com.fasterxml.jackson.core
- jackson-core
-
-
- com.fasterxml.jackson.core
- jackson-annotations
-
-
- protobuf-java
- com.google.protobuf
-
-
- compile
-
org.scalacheck
scalacheck_${scala.binary.version}
@@ -331,9 +240,6 @@
-
- ${cpp.build.dir}
-
${resource.dir}
@@ -349,69 +255,6 @@
-
- exec-maven-plugin
- org.codehaus.mojo
- 1.6.0
-
-
- Build arrow
- generate-resources
-
- exec
-
-
- bash
-
- ${arrow.script.dir}/build_arrow.sh
- --tests=${cpp_tests}
- --build_arrow=${build_arrow}
- --static_arrow=${static_arrow}
- --arrow_root=${arrow_root}
-
-
-
-
- Build cpp
- generate-resources
-
- exec
-
-
- bash
-
- ${cpp.dir}/compile.sh
- ${build_cpp}
- ${cpp_tests}
- ${jvm.build_arrow}
- ${static_arrow}
- ${build_protobuf}
- ${arrow_root}
- ${jvm.arrow.bfs.install.dir}
- ${build_jemalloc}
- ${build_gazelle_cpp}
- ${build_velox}
- ${velox_home}
-
-
-
-
- Build velox
- generate-resources
-
- exec
-
-
- bash
-
- ${velox.script.dir}/build_velox.sh
- --build_velox_from_source=${build_velox_from_source}
-
-
-
-
-
-
org.apache.maven.plugins
@@ -549,24 +392,6 @@
-
- maven-assembly-plugin
- 3.3.0
-
-
- jar-with-dependencies
-
-
-
-
- make-assembly
- package
-
- single
-
-
-
-
org.scalastyle
scalastyle-maven-plugin
diff --git a/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java b/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java
new file mode 100644
index 000000000000..2a3d7e028036
--- /dev/null
+++ b/jvm/src/main/java/io/glutenproject/execution/AbstractColumnarNativeIterator.java
@@ -0,0 +1,48 @@
+/*
+ * 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 io.glutenproject.execution;
+
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+
+import java.util.Iterator;
+
+abstract public class AbstractColumnarNativeIterator implements Iterator, AutoCloseable {
+ protected final Iterator delegated;
+ protected ColumnarBatch nextBatch = null;
+
+ public AbstractColumnarNativeIterator(Iterator delegated) {
+ this.delegated = delegated;
+ }
+
+ @Override
+ public boolean hasNext() {
+ while (delegated.hasNext()) {
+ nextBatch = delegated.next();
+ if (nextBatch.numRows() > 0) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ public abstract byte[] next();
+
+ @Override
+ public void close() throws Exception {
+ }
+}
diff --git a/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java b/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java
deleted file mode 100644
index 65af1c81dafb..000000000000
--- a/jvm/src/main/java/io/glutenproject/execution/CHColumnarNativeIterator.java
+++ /dev/null
@@ -1,40 +0,0 @@
-package io.glutenproject.execution;
-
-import io.glutenproject.expression.ConverterUtils;
-import io.glutenproject.vectorized.CHColumnVector;
-import io.glutenproject.vectorized.CHNativeBlock;
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-import java.util.Iterator;
-
-public class CHColumnarNativeIterator extends ColumnarNativeIterator {
-
- public CHColumnarNativeIterator(Iterator delegated) {
- super(delegated);
- }
-
- @Override
- public byte[] next() {
- ColumnarBatch dep_cb = nextBatch;
- if (dep_cb.numRows() > 0) {
- CHColumnVector col = (CHColumnVector)dep_cb.column(0);
- return longtoBytes(col.getBlockAddress());
- } else {
- throw new IllegalStateException();
- }
- }
-
- private static byte[] longtoBytes(long data) {
- return new byte[]{
- (byte) ((data >> 56) & 0xff),
- (byte) ((data >> 48) & 0xff),
- (byte) ((data >> 40) & 0xff),
- (byte) ((data >> 32) & 0xff),
- (byte) ((data >> 24) & 0xff),
- (byte) ((data >> 16) & 0xff),
- (byte) ((data >> 8) & 0xff),
- (byte) ((data >> 0) & 0xff),
- };
- }
-}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java b/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java
new file mode 100644
index 000000000000..4df588e2aaa2
--- /dev/null
+++ b/jvm/src/main/java/io/glutenproject/vectorized/AbstractBatchIterator.java
@@ -0,0 +1,66 @@
+/*
+ * 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 io.glutenproject.vectorized;
+
+import java.io.IOException;
+
+import java.io.Serializable;
+
+abstract public class AbstractBatchIterator implements AutoCloseable, Serializable {
+
+ protected long nativeHandler = 0;
+ protected boolean closed = false;
+
+ public AbstractBatchIterator() throws IOException {}
+
+ public AbstractBatchIterator(long instance_id) throws IOException {
+ nativeHandler = instance_id;
+ }
+
+ public abstract boolean hasNextInternal() throws IOException;
+
+ public boolean hasNext() throws IOException {
+ return hasNextInternal();
+ }
+
+ public abstract T nextInternal() throws IOException;
+
+ public T next() throws IOException {
+ return nextInternal();
+ }
+
+ public abstract MetricsObject getMetricsInternal() throws IOException, ClassNotFoundException;
+
+ public MetricsObject getMetrics() throws IOException, ClassNotFoundException {
+ if (nativeHandler == 0) {
+ return null;
+ }
+ return getMetricsInternal();
+ }
+
+ public abstract void closeInternal();
+
+ @Override
+ public void close() {
+ closeInternal();
+ }
+
+ long getInstanceId() {
+ return nativeHandler;
+ }
+}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java b/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java
deleted file mode 100644
index 7a399640bf36..000000000000
--- a/jvm/src/main/java/io/glutenproject/vectorized/CHNativeBlock.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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 io.glutenproject.vectorized;
-
-import org.apache.spark.sql.types.DataType;
-import org.apache.spark.sql.vectorized.ColumnVector;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-
-import java.util.Optional;
-import java.util.stream.IntStream;
-
-public class CHNativeBlock {
- private long blockAddress;
-
- public CHNativeBlock(long blockAddress) {
- this.blockAddress = blockAddress;
- }
-
- public static Optional fromColumnarBatch(ColumnarBatch batch) {
- if (batch.numCols() == 0 || !(batch.column(0) instanceof CHColumnVector)) {
- return Optional.empty();
- }
- CHColumnVector columnVector = (CHColumnVector) batch.column(0);
- return Optional.of(new CHNativeBlock(columnVector.getBlockAddress()));
- }
-
- private native int nativeNumRows(long blockAddress);
-
- public int numRows() {
- return nativeNumRows(blockAddress);
- };
-
- private native int nativeNumColumns(long blockAddress);
-
- public int numColumns() {
- return nativeNumColumns(blockAddress);
- }
-
- private native String nativeColumnType(long blockAddress, int position);
-
- public String getTypeByPosition(int position) {
- return nativeColumnType(blockAddress, position);
- }
-
- private native long nativeTotalBytes(long blockAddress);
-
- public long totalBytes() {
- return nativeTotalBytes(blockAddress);
- }
-
- public native void nativeClose(long blockAddress);
-
- public void close() {
- if (blockAddress != 0) {
- nativeClose(blockAddress);
- blockAddress = 0;
- }
- }
-
- public ColumnarBatch toColumnarBatch() {
- ColumnVector[] vectors = new ColumnVector[numColumns()];
- for (int i = 0; i < numColumns(); i++) {
- vectors[i] = new CHColumnVector(DataType.fromDDL(getTypeByPosition(i)), blockAddress, i);
- }
- int numRows = 0;
- if (numColumns() != 0) {
- numRows = numRows();
- }
- return new ColumnarBatch(vectors, numRows);
- }
-}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java b/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java
deleted file mode 100644
index 1d5bedda2c95..000000000000
--- a/jvm/src/main/java/io/glutenproject/vectorized/ColumnarFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-package io.glutenproject.vectorized;
-
-import io.glutenproject.GlutenConfig;
-import org.apache.spark.serializer.Serializer;
-import org.apache.spark.sql.execution.metric.SQLMetric;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.sql.vectorized.ColumnarBatch;
-import scala.collection.Iterator;
-
-import java.io.IOException;
-
-
-public class ColumnarFactory {
- private ColumnarFactory() {
- }
-
- public static boolean isEnableCH() {
- return GlutenConfig.getConf().isClickHouseBackend();
- }
-
- public static Iterator createClosableIterator(Iterator iter) {
- if (isEnableCH()) {
- return new CloseableCHColumnBatchIterator(iter);
- } else {
- return new CloseableColumnBatchIterator(iter);
- }
- }
-
- public static Object createShuffleSplitterJniWrapper() throws IOException {
- if (isEnableCH()) {
- return new CHShuffleSplitterJniWrapper();
- } else {
- return new ShuffleSplitterJniWrapper();
- }
- }
-
- public static Serializer createColumnarBatchSerializer(StructType schema, SQLMetric readBatchNumRows, SQLMetric numOutputRows) {
- if (isEnableCH()) {
- return new CHColumnarBatchSerializer(readBatchNumRows, numOutputRows);
- } else {
- return new ArrowColumnarBatchSerializer(schema, readBatchNumRows, numOutputRows);
- }
- }
-}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java
index 8f34f6c61339..a090d48ea7d9 100644
--- a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java
+++ b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluator.java
@@ -18,23 +18,14 @@
package io.glutenproject.vectorized;
import io.glutenproject.GlutenConfig;
-import io.glutenproject.execution.ColumnarNativeIterator;
+import io.glutenproject.backendsapi.BackendsApiManager;
+import io.glutenproject.execution.AbstractColumnarNativeIterator;
import io.glutenproject.row.RowIterator;
import io.glutenproject.substrait.plan.PlanNode;
-import org.apache.arrow.dataset.jni.NativeMemoryPool;
-import org.apache.arrow.gandiva.exceptions.GandivaException;
-import org.apache.arrow.gandiva.expression.ExpressionTree;
-import org.apache.arrow.gandiva.ipc.GandivaTypes;
-import org.apache.arrow.vector.ipc.WriteChannel;
-import org.apache.arrow.vector.ipc.message.MessageSerializer;
-import org.apache.arrow.vector.types.pojo.Schema;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.spark.memory.MemoryConsumer;
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils;
-
-import java.io.ByteArrayOutputStream;
+
+import scala.collection.JavaConverters;
+
import java.io.IOException;
-import java.nio.channels.Channels;
import java.util.ArrayList;
import java.util.List;
@@ -95,25 +86,27 @@ public boolean doValidate(byte[] subPlan) {
// Used by WholeStageTransfrom to create the native computing pipeline and
// return a columnar result iterator.
- public BatchIterator createKernelWithBatchIterator(
- byte[] wsPlan, ArrayList iterList)
+ public AbstractBatchIterator createKernelWithBatchIterator(
+ byte[] wsPlan, ArrayList iterList)
throws RuntimeException, IOException {
- long poolId = 0;
+ /* long poolId = 0;
if (!GlutenConfig.getConf().isClickHouseBackend()) {
- NativeMemoryPool memoryPool = SparkMemoryUtils.contextMemoryPool();
- poolId = memoryPool.getNativeInstanceId();
+ // NativeMemoryPool memoryPool = SparkMemoryUtils.contextMemoryPool();
+ // poolId = memoryPool.getNativeInstanceId();
}
ColumnarNativeIterator[] iterArray = new ColumnarNativeIterator[iterList.size()];
long batchIteratorInstance = jniWrapper.nativeCreateKernelWithIterator(
- poolId, wsPlan, iterList.toArray(iterArray));
- return new BatchIterator(batchIteratorInstance);
+ poolId, wsPlan, iterList.toArray(iterArray)); */
+ return BackendsApiManager.getIteratorApiInstance()
+ .genBatchIterator(wsPlan,
+ JavaConverters.asScalaIteratorConverter(iterList.iterator()).asScala().toSeq(),
+ jniWrapper);
}
-
// Used by WholeStageTransfrom to create the native computing pipeline and
// return a columnar result iterator.
- public BatchIterator createKernelWithBatchIterator(
- PlanNode wsPlan, ArrayList iterList)
+ public AbstractBatchIterator createKernelWithBatchIterator(
+ PlanNode wsPlan, ArrayList iterList)
throws RuntimeException, IOException {
return createKernelWithBatchIterator(getPlanBytesBuf(wsPlan), iterList);
}
@@ -122,7 +115,7 @@ public BatchIterator createKernelWithBatchIterator(
// return a row result iterator.
public RowIterator createKernelWithRowIterator(
byte[] wsPlan,
- ArrayList iterList) throws RuntimeException, IOException {
+ ArrayList iterList) throws RuntimeException, IOException {
long rowIteratorInstance = jniWrapper.nativeCreateKernelWithRowIterator(wsPlan);
return new RowIterator(rowIteratorInstance);
}
@@ -132,21 +125,6 @@ public void close() {
jniWrapper.nativeClose(nativeHandler);
}
- byte[] getSchemaBytesBuf(Schema schema) throws IOException {
- if (schema == null) return null;
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema);
- return out.toByteArray();
- }
-
- byte[] getExprListBytesBuf(List exprs) throws GandivaException {
- GandivaTypes.ExpressionList.Builder builder = GandivaTypes.ExpressionList.newBuilder();
- for (ExpressionTree expr : exprs) {
- builder.addExprs(expr.toProtobuf());
- }
- return builder.build().toByteArray();
- }
-
byte[] getPlanBytesBuf(PlanNode planNode) {
return planNode.toProtobuf().toByteArray();
}
diff --git a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java
index 129c85e3acc2..13d8f90343b9 100644
--- a/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java
+++ b/jvm/src/main/java/io/glutenproject/vectorized/ExpressionEvaluatorJniWrapper.java
@@ -17,8 +17,7 @@
package io.glutenproject.vectorized;
-import io.glutenproject.execution.ColumnarNativeIterator;
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
+import io.glutenproject.execution.AbstractColumnarNativeIterator;
import org.apache.spark.memory.MemoryConsumer;
import java.io.IOException;
@@ -63,9 +62,10 @@ public ExpressionEvaluatorJniWrapper(String tmp_dir, List listJars, Stri
* @param nativeHandler nativeHandler of this expression
* @return iterator instance id
*/
- native long nativeCreateKernelWithIterator(long nativeHandler,
+ public native long nativeCreateKernelWithIterator(long nativeHandler,
byte[] wsPlan,
- ColumnarNativeIterator[] batchItr) throws RuntimeException;
+ AbstractColumnarNativeIterator[] batchItr
+ ) throws RuntimeException;
/**
* Create a native compute kernel and return a row iterator.
diff --git a/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala b/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala
index 49cc29cb9603..d3793f933eba 100644
--- a/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala
+++ b/jvm/src/main/scala/io/glutenproject/GlutenConfig.scala
@@ -188,6 +188,9 @@ class GlutenConfig(conf: SQLConf) extends Logging {
val isClickHouseBackend: Boolean =
glutenBackendLib.equalsIgnoreCase(GlutenConfig.GLUTEN_CLICKHOUSE_BACKEND)
+ val isGazelleBackend: Boolean =
+ glutenBackendLib.equalsIgnoreCase(GlutenConfig.GLUTEN_GAZELLE_BACKEND)
+
// fallback to row operators if there are several continous joins
val joinOptimizationThrottle: Integer =
conf.getConfString("spark.gluten.sql.columnar.joinOptimizationLevel", "12").toInt
@@ -255,7 +258,8 @@ object GlutenConfig {
val GLUTEN_LOAD_ARROW = "spark.gluten.sql.columnar.loadarrow"
val GLUTEN_BACKEND_LIB = "spark.gluten.sql.columnar.backend.lib"
val GLUTEN_VELOX_BACKEND = "velox"
- val GLUTEN_CLICKHOUSE_BACKEND = "clickhouse"
+ val GLUTEN_CLICKHOUSE_BACKEND = "ch"
+ val GLUTEN_GAZELLE_BACKEND = "gazelle_cpp"
var ins: GlutenConfig = null
var random_temp_dir_path: String = null
diff --git a/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala b/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala
index 3a267ce92d10..7f65f9a98c6c 100644
--- a/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala
+++ b/jvm/src/main/scala/io/glutenproject/GlutenPlugin.scala
@@ -22,6 +22,7 @@ import java.util.{Collections, Objects}
import scala.language.implicitConversions
import io.glutenproject.GlutenPlugin.{GLUTEN_SESSION_EXTENSION_NAME, SPARK_SESSION_EXTS_KEY}
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.extension.{ColumnarOverrides, StrategyOverrides}
import io.glutenproject.vectorized.ExpressionEvaluator
import java.util
@@ -45,6 +46,9 @@ private[glutenproject] class GlutenDriverPlugin extends DriverPlugin {
override def init(sc: SparkContext, pluginContext: PluginContext): util.Map[String, String] = {
val conf = pluginContext.conf()
setPredefinedConfigs(conf)
+ // Initialize Backends API
+ BackendsApiManager.initialize(pluginContext.conf()
+ .get(GlutenConfig.GLUTEN_BACKEND_LIB, ""))
Collections.emptyMap()
}
@@ -64,17 +68,18 @@ private[glutenproject] class GlutenExecutorPlugin extends ExecutorPlugin {
override def init(ctx: PluginContext, extraConf: util.Map[String, String]): Unit = {
// SQLConf is not initialed here, so it can not use 'GlutenConfig.getConf' to get conf.
if (ctx.conf().getBoolean(GlutenConfig.GLUTEN_LOAD_NATIVE, defaultValue = true)) {
- val customGLUTENLib = ctx.conf().get(GlutenConfig.GLUTEN_LIB_PATH, "")
+ val customGlutenLib = ctx.conf().get(GlutenConfig.GLUTEN_LIB_PATH, "")
val customBackendLib = ctx.conf().get(GlutenConfig.GLUTEN_BACKEND_LIB, "")
val initKernel = new ExpressionEvaluator(java.util.Collections.emptyList[String],
ctx.conf().get(GlutenConfig.GLUTEN_LIB_NAME, "spark_columnar_jni"),
- customGLUTENLib,
+ customGlutenLib,
customBackendLib,
ctx.conf().getBoolean(GlutenConfig.GLUTEN_LOAD_ARROW, defaultValue = true))
- if (customGLUTENLib.nonEmpty || customBackendLib.nonEmpty) {
+ if (customGlutenLib.nonEmpty || customBackendLib.nonEmpty) {
initKernel.initNative()
}
}
+ BackendsApiManager.initialize(ctx.conf().get(GlutenConfig.GLUTEN_BACKEND_LIB, ""))
}
/**
diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala
new file mode 100644
index 000000000000..a2c2751b257d
--- /dev/null
+++ b/jvm/src/main/scala/io/glutenproject/backendsapi/BackendsApiManager.scala
@@ -0,0 +1,98 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi
+
+import java.util.ServiceLoader
+import scala.collection.JavaConversions._
+
+object BackendsApiManager {
+
+ protected var iteratorApiInstance: IIteratorApi = null
+
+ protected var sparkPlanExecApiInstance: ISparkPlanExecApi = null
+
+ protected var transformerApiInstance: ITransformerApi = null
+
+ /**
+ * Initialize all backends api.
+ *
+ * @param glutenBackenLibName
+ */
+ def initialize(glutenBackenLibName: String): Unit = synchronized {
+ // initialize IIteratorApi instance
+ if (iteratorApiInstance == null) {
+ val serviceLoader = ServiceLoader.load(classOf[IIteratorApi])
+ for (ele <- serviceLoader) {
+ if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) {
+ iteratorApiInstance = ele
+ } else if (ele.getBackendName.equalsIgnoreCase("velox")) {
+ // Currently, Velox and Gazelle backends use the same api to implement.
+ // Will remove this after splitting gazelle module
+ iteratorApiInstance = ele
+ }
+ }
+ }
+ assert(iteratorApiInstance != null, "Can not initialize IIteratorApi instance.")
+ // initialize ISparkPlanExecApi instance
+ if (sparkPlanExecApiInstance == null) {
+ val serviceLoader = ServiceLoader.load(classOf[ISparkPlanExecApi])
+
+ for (ele <- serviceLoader) {
+ if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) {
+ sparkPlanExecApiInstance = ele
+ } else if (ele.getBackendName.equalsIgnoreCase("velox")) {
+ // Currently, Velox and Gazelle backends use the same api to implement.
+ // Will remove this after splitting gazelle module
+ sparkPlanExecApiInstance = ele
+ }
+ }
+ }
+ assert(sparkPlanExecApiInstance != null, "Can not initialize " + "ISparkPlanExecApi instance.")
+ // initialize ITransformerApi instance
+ if (transformerApiInstance == null) {
+ val serviceLoader = ServiceLoader.load(classOf[ITransformerApi])
+ for (ele <- serviceLoader) {
+ if (ele.getBackendName.equalsIgnoreCase(glutenBackenLibName)) {
+ transformerApiInstance = ele
+ } else if (ele.getBackendName.equalsIgnoreCase("velox")) {
+ // Currently, Velox and Gazelle backends use the same api to implement.
+ // Will remove this after splitting gazelle module
+ transformerApiInstance = ele
+ }
+ }
+ }
+ assert(transformerApiInstance != null, "Can not initialize ITransformerApi instance.")
+ }
+
+ def getIteratorApiInstance: IIteratorApi = {
+ if (iteratorApiInstance == null)
+ throw new RuntimeException("IIteratorApi instance is null.")
+ iteratorApiInstance
+ }
+
+ def getSparkPlanExecApiInstance: ISparkPlanExecApi = {
+ if (sparkPlanExecApiInstance == null)
+ throw new RuntimeException("ISparkPlanExecApi instance is null.")
+ sparkPlanExecApiInstance
+ }
+
+ def getTransformerApiInstance: ITransformerApi = {
+ if (transformerApiInstance == null)
+ throw new RuntimeException("ITransformerApi instance is null.")
+ transformerApiInstance
+ }
+}
diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala
new file mode 100644
index 000000000000..19328caf5bd1
--- /dev/null
+++ b/jvm/src/main/scala/io/glutenproject/backendsapi/IBackendsApi.scala
@@ -0,0 +1,27 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi
+
+trait IBackendsApi {
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ def getBackendName: String
+}
diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala
new file mode 100644
index 000000000000..5d6baa29ffa6
--- /dev/null
+++ b/jvm/src/main/scala/io/glutenproject/backendsapi/IIteratorApi.scala
@@ -0,0 +1,97 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi
+
+import io.glutenproject.GlutenNumaBindingInfo
+import io.glutenproject.execution.{AbstractColumnarNativeIterator, BaseNativeFilePartition, WholestageTransformContext}
+import io.glutenproject.substrait.plan.PlanNode
+import io.glutenproject.vectorized.{AbstractBatchIterator, ExpressionEvaluator, ExpressionEvaluatorJniWrapper}
+import org.apache.spark.{SparkConf, TaskContext}
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+trait IIteratorApi extends IBackendsApi {
+
+ /**
+ * Generate native row partition.
+ *
+ * @return
+ */
+ def genNativeFilePartition(p: InputPartition,
+ wsCxt: WholestageTransformContext): BaseNativeFilePartition
+
+ /**
+ * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec.
+ *
+ * @return
+ */
+ def genCoalesceIterator(iter: Iterator[ColumnarBatch],
+ recordsPerBatch: Int, numOutputRows: SQLMetric,
+ numInputBatches: SQLMetric, numOutputBatches: SQLMetric,
+ collectTime: SQLMetric, concatTime: SQLMetric,
+ avgCoalescedNumRows: SQLMetric): Iterator[ColumnarBatch]
+
+ /**
+ * Generate closeable ColumnBatch iterator.
+ *
+ * @return
+ */
+ def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch]
+
+ /**
+ * Generate Iterator[ColumnarBatch] for first stage.
+ *
+ * @return
+ */
+ def genFirstStageIterator(inputPartition: BaseNativeFilePartition, loadNative: Boolean,
+ outputAttributes: Seq[Attribute], context: TaskContext,
+ jarList: Seq[String]): Iterator[ColumnarBatch]
+
+ /**
+ * Generate Iterator[ColumnarBatch] for final stage.
+ *
+ * @return
+ */
+ def genFinalStageIterator(iter: Iterator[ColumnarBatch],
+ numaBindingInfo: GlutenNumaBindingInfo, listJars: Seq[String],
+ signature: String, sparkConf: SparkConf,
+ outputAttributes: Seq[Attribute], rootNode: PlanNode,
+ streamedSortPlan: SparkPlan, pipelineTime: SQLMetric,
+ buildRelationBatchHolder: Seq[ColumnarBatch],
+ dependentKernels: Seq[ExpressionEvaluator],
+ dependentKernelIterators: Seq[AbstractBatchIterator]
+ ): Iterator[ColumnarBatch]
+
+ /**
+ * Generate columnar native iterator.
+ *
+ * @return
+ */
+ def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch]): AbstractColumnarNativeIterator
+
+ /**
+ * Generate BatchIterator for ExpressionEvaluator.
+ *
+ * @return
+ */
+ def genBatchIterator(wsPlan: Array[Byte], iterList: Seq[AbstractColumnarNativeIterator],
+ jniWrapper: ExpressionEvaluatorJniWrapper): AbstractBatchIterator
+}
diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala
new file mode 100644
index 000000000000..6d268c8daa19
--- /dev/null
+++ b/jvm/src/main/scala/io/glutenproject/backendsapi/ISparkPlanExecApi.scala
@@ -0,0 +1,78 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi
+
+import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec}
+
+import org.apache.spark.ShuffleDependency
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+trait ISparkPlanExecApi extends IBackendsApi {
+
+ /**
+ * Generate NativeColumnarToRowExec.
+ *
+ * @param child
+ * @return
+ */
+ def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec
+
+ /**
+ * Generate RowToArrowColumnarExec.
+ *
+ * @param child
+ * @return
+ */
+ def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec
+
+ /**
+ * Generate ShuffleDependency for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ def genShuffleDependency(rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute],
+ newPartitioning: Partitioning, serializer: Serializer,
+ writeMetrics: Map[String, SQLMetric], dataSize: SQLMetric,
+ bytesSpilled: SQLMetric, numInputRows: SQLMetric,
+ computePidTime: SQLMetric, splitTime: SQLMetric,
+ spillTime: SQLMetric, compressTime: SQLMetric, prepareTime: SQLMetric
+ ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch]
+
+ /**
+ * Generate ColumnarShuffleWriter for ColumnarShuffleManager.
+ *
+ * @return
+ */
+ def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V]
+ ): GlutenShuffleWriterWrapper[K, V]
+
+ /**
+ * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ def createColumnarBatchSerializer(schema: StructType, readBatchNumRows: SQLMetric,
+ numOutputRows: SQLMetric): Serializer
+}
diff --git a/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala b/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala
new file mode 100644
index 000000000000..1d2a5fbe830d
--- /dev/null
+++ b/jvm/src/main/scala/io/glutenproject/backendsapi/ITransformerApi.scala
@@ -0,0 +1,31 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package io.glutenproject.backendsapi
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+
+trait ITransformerApi extends IBackendsApi {
+
+ /**
+ * Do validate for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning,
+ outputAttributes: Seq[Attribute]): Boolean
+}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala
index 6ea4d17d5877..5e409da56a39 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/BroadcastHashJoinExecTransformer.scala
@@ -17,33 +17,20 @@
package io.glutenproject.execution
-import com.google.common.collect.Lists
-import io.glutenproject.GlutenConfig
-import io.glutenproject.expression._
import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.substrait.rel.RelNode
-import io.glutenproject.vectorized.{ExpressionEvaluator, _}
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.vector.types.pojo.{ArrowType, Field}
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.joins.{BaseJoinExec, HashJoin, ShuffledJoin}
-import org.apache.spark.sql.execution.joins.HashedRelationInfo
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide}
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, PartitioningCollection}
import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils}
-import org.apache.spark.sql.types.DecimalType
+import org.apache.spark.sql.vectorized.ColumnarBatch
-import scala.collection.JavaConverters._
import scala.collection.mutable
-import scala.collection.mutable.ListBuffer
/**
* Performs a hash join of two child relations by first shuffling the data using the join keys.
diff --git a/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala b/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala
index 1967b3b50d8c..b444fcbe5228 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/CoalesceBatchesExec.scala
@@ -17,24 +17,15 @@
package io.glutenproject.execution
-import scala.collection.mutable.ListBuffer
-import scala.collection.JavaConverters._
-
-import io.glutenproject.GlutenConfig
-import io.glutenproject.vectorized.{ArrowWritableColumnVector, CHCoalesceOperator, CHNativeBlock, ColumnarFactory}
-import org.apache.arrow.vector.types.pojo.Schema
-import org.apache.arrow.vector.util.VectorBatchAppender
-import org.apache.spark.TaskContext
+import io.glutenproject.backendsapi.BackendsApiManager
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
+import org.apache.spark.sql.vectorized.ColumnarBatch
case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode {
@@ -58,16 +49,6 @@ case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode {
.createAverageMetric(sparkContext, "avg coalesced batch num rows"))
override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- if (GlutenConfig.getConf.isClickHouseBackend) {
- doCHInternalExecuteColumnar()
- } else {
- doInternalExecuteColumnar()
- }
- }
-
- def doInternalExecuteColumnar(): RDD[ColumnarBatch] = {
- import CoalesceBatchesExec._
-
val recordsPerBatch = conf.arrowMaxRecordsPerBatch
val numOutputRows = longMetric("numOutputRows")
val numInputBatches = longMetric("numInputBatches")
@@ -77,142 +58,9 @@ case class CoalesceBatchesExec(child: SparkPlan) extends UnaryExecNode {
val avgCoalescedNumRows = longMetric("avgCoalescedNumRows")
child.executeColumnar().mapPartitions { iter =>
- val beforeInput = System.nanoTime
- val hasInput = iter.hasNext
- collectTime += System.nanoTime - beforeInput
- val res = if (hasInput) {
- new Iterator[ColumnarBatch] {
- var numBatchesTotal: Long = _
- var numRowsTotal: Long = _
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ =>
- if (numBatchesTotal > 0) {
- avgCoalescedNumRows.set(numRowsTotal.toDouble / numBatchesTotal)
- }
- }
-
- override def hasNext: Boolean = {
- val beforeNext = System.nanoTime
- val hasNext = iter.hasNext
- collectTime += System.nanoTime - beforeNext
- hasNext
- }
-
- override def next(): ColumnarBatch = {
-
- if (!hasNext) {
- throw new NoSuchElementException("End of ColumnarBatch iterator")
- }
-
- var rowCount = 0
- val batchesToAppend = ListBuffer[ColumnarBatch]()
-
- while (hasNext && rowCount < recordsPerBatch) {
- val delta = iter.next()
- delta.retain()
- rowCount += delta.numRows
- batchesToAppend += delta
- }
-
- // chendi: We need make sure target FieldTypes are exactly the same as src
- val expected_output_arrow_fields = if (batchesToAppend.size > 0) {
- (0 until batchesToAppend(0).numCols).map(i => {
- batchesToAppend(0).column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector.getField
- })
- } else {
- Nil
- }
-
- val resultStructType = ArrowUtils.fromArrowSchema(new Schema(expected_output_arrow_fields.asJava))
- val beforeConcat = System.nanoTime
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(rowCount, resultStructType).toArray
- val target =
- new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), rowCount)
- coalesce(target, batchesToAppend.toList)
- target.setNumRows(rowCount)
-
- concatTime += System.nanoTime - beforeConcat
- numOutputRows += rowCount
- numInputBatches += batchesToAppend.length
- numOutputBatches += 1
-
- // used for calculating avgCoalescedNumRows
- numRowsTotal += rowCount
- numBatchesTotal += 1
-
- batchesToAppend.foreach(cb => cb.close())
-
- target
- }
- }
- } else {
- Iterator.empty
- }
- ColumnarFactory.createClosableIterator(res)
- }
- }
-
- def doCHInternalExecuteColumnar(): RDD[ColumnarBatch] = {
- val recordsPerBatch = conf.arrowMaxRecordsPerBatch
- val numOutputRows = longMetric("numOutputRows")
- val numInputBatches = longMetric("numInputBatches")
- val numOutputBatches = longMetric("numOutputBatches")
- val collectTime = longMetric("collectTime")
- val concatTime = longMetric("concatTime")
- val avgCoalescedNumRows = longMetric("avgCoalescedNumRows")
- child.executeColumnar().mapPartitions(iter => {
- val operator = new CHCoalesceOperator(recordsPerBatch)
- val res = new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = {
- val beforeNext = System.nanoTime
- val hasNext = iter.hasNext
- collectTime += System.nanoTime - beforeNext
- hasNext
- }
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit]((tc: TaskContext) => {
- operator.close()
- })
- override def next(): ColumnarBatch = {
- val c = iter.next()
- numInputBatches += 1
- val beforeConcat = System.nanoTime
- operator.mergeBlock(c)
-
- while(!operator.isFull && iter.hasNext) {
- val cb = iter.next();
- numInputBatches += 1;
- operator.mergeBlock(cb)
- }
- val res = operator.release().toColumnarBatch
- CHNativeBlock.fromColumnarBatch(res).ifPresent(block => {
- numOutputRows += block.numRows();
- numOutputBatches += 1;
- })
- res
- }
- }
-
- ColumnarFactory.createClosableIterator(res)
- })
- }
-}
-
-object CoalesceBatchesExec {
- implicit class ArrowColumnarBatchRetainer(val cb: ColumnarBatch) {
- def retain(): Unit = {
- (0 until cb.numCols).toList.foreach(i =>
- cb.column(i).asInstanceOf[ArrowWritableColumnVector].retain())
- }
- }
-
- def coalesce(targetBatch: ColumnarBatch, batchesToAppend: List[ColumnarBatch]): Unit = {
- (0 until targetBatch.numCols).toList.foreach { i =>
- val targetVector =
- targetBatch.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector
- val vectorsToAppend = batchesToAppend.map { cb =>
- cb.column(i).asInstanceOf[ArrowWritableColumnVector].getValueVector
- }
- VectorBatchAppender.batchAppend(targetVector, vectorsToAppend: _*)
+ BackendsApiManager.getIteratorApiInstance
+ .genCoalesceIterator(iter, recordsPerBatch, numOutputRows,
+ numInputBatches, numOutputBatches, collectTime, concatTime, avgCoalescedNumRows)
}
}
}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala
index b01f70fddd26..369cc06c1075 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala
@@ -17,22 +17,15 @@
package io.glutenproject.execution
-import io.glutenproject.expression._
import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized._
-import org.apache.arrow.vector.ValueVector
-import org.apache.spark.internal.Logging
+
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-import org.apache.spark.sql.types.{StructField, StructType}
-import org.apache.spark.TaskContext
-import org.apache.spark.sql.types.DecimalType
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.vectorized.ColumnarBatch
case class ExpandExecTransformer(
projections: Seq[Seq[Expression]],
diff --git a/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala
index 80641fc801cf..ef4f0ce1dc57 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala
@@ -18,19 +18,21 @@
package io.glutenproject.execution
import scala.collection.mutable.ListBuffer
+import scala.util.control.Breaks.{break, breakable}
+
import com.google.common.collect.Lists
+import com.google.protobuf.Any
import io.glutenproject.expression._
import io.glutenproject.substrait.expression.{AggregateFunctionNode, ExpressionBuilder, ExpressionNode}
import io.glutenproject.substrait.rel.{LocalFilesBuilder, RelBuilder, RelNode}
import io.glutenproject.substrait.SubstraitContext
import io.glutenproject.GlutenConfig
-import java.util
-
-import com.google.protobuf.Any
import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode}
import io.glutenproject.substrait.extensions.ExtensionBuilder
import io.glutenproject.substrait.plan.PlanBuilder
import io.glutenproject.vectorized.ExpressionEvaluator
+import java.util
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -41,8 +43,6 @@ import org.apache.spark.sql.execution.aggregate._
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.vectorized.ColumnarBatch
-import scala.util.control.Breaks.{break, breakable}
-
/**
* Columnar Based HashAggregateExec.
*/
@@ -94,6 +94,15 @@ case class HashAggregateExecTransformer(
numOutputBatches.set(0)
numInputBatches.set(0)
+ lazy val aggregateResultAttributes = {
+ val groupingAttributes = groupingExpressions.map(expr => {
+ ConverterUtils.getAttrFromExpr(expr).toAttribute
+ })
+ groupingAttributes ++ aggregateExpressions.map(expr => {
+ expr.resultAttribute
+ })
+ }
+
override def doValidate(): Boolean = {
val substraitContext = new SubstraitContext
val relNode = try {
@@ -167,20 +176,41 @@ case class HashAggregateExecTransformer(
case _ =>
null
}
- val (relNode, inputAttributes) = if (childCtx != null) {
- (getAggRel(context.registeredFunction, childCtx.root), childCtx.outputAttributes)
+ val (relNode, inputAttributes, outputAttributes) = if (childCtx != null) {
+ if (!GlutenConfig.getConf.isClickHouseBackend) {
+ (getAggRel(context.registeredFunction, childCtx.root),
+ childCtx.outputAttributes, output)
+ } else {
+ (getAggRel(context.registeredFunction, childCtx.root),
+ childCtx.outputAttributes, aggregateResultAttributes)
+ }
} else {
// This means the input is just an iterator, so an ReadRel will be created as child.
// Prepare the input schema.
- val attrList = new util.ArrayList[Attribute]()
- for (attr <- child.output) {
- attrList.add(attr)
- }
- val readRel = RelBuilder.makeReadRel(attrList, context)
+ if (!GlutenConfig.getConf.isClickHouseBackend) {
+ val attrList = new util.ArrayList[Attribute]()
+ for (attr <- child.output) {
+ attrList.add(attr)
+ }
+ val readRel = RelBuilder.makeReadRel(attrList, context)
+ (getAggRel(context.registeredFunction, readRel), child.output, output)
+ } else {
+ val typeList = new util.ArrayList[TypeNode]()
+ val nameList = new util.ArrayList[String]()
+ for (attr <- aggregateResultAttributes) {
+ typeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable))
+ nameList.add(ConverterUtils.getShortAttributeName(attr) + "#" + attr.exprId.id)
+ }
+ // The iterator index will be added in the path of LocalFiles.
+ val inputIter = LocalFilesBuilder.makeLocalFiles(
+ ConverterUtils.ITERATOR_PREFIX.concat(context.getIteratorIndex.toString))
+ context.setLocalFilesNode(inputIter)
+ val readRel = RelBuilder.makeReadRel(typeList, nameList, context)
- (getAggRel(context.registeredFunction, readRel), child.output)
+ (getAggRel(context.registeredFunction, readRel), aggregateResultAttributes, output)
+ }
}
- TransformContext(inputAttributes, output, relNode)
+ TransformContext(inputAttributes, outputAttributes, relNode)
}
override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields)
@@ -362,11 +392,17 @@ case class HashAggregateExecTransformer(
aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args)
})
case Final =>
- aggregatFunc.inputAggBufferAttributes.toList.map(attr => {
- val aggExpr: Expression = ExpressionConverter
- .replaceWithExpressionTransformer(attr, originalInputAttributes)
- aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args)
- })
+ if (!GlutenConfig.getConf.isClickHouseBackend) {
+ aggregatFunc.inputAggBufferAttributes.toList.map(attr => {
+ val aggExpr: Expression = ExpressionConverter
+ .replaceWithExpressionTransformer(attr, originalInputAttributes)
+ aggExpr.asInstanceOf[ExpressionTransformer].doTransform(args)
+ })
+ } else {
+ val aggTypesExpr: Expression = ExpressionConverter
+ .replaceWithExpressionTransformer(aggExpr.resultAttribute, originalInputAttributes)
+ Seq(aggTypesExpr.asInstanceOf[ExpressionTransformer].doTransform(args))
+ }
case other =>
throw new UnsupportedOperationException(s"$other not supported.")
}
@@ -400,7 +436,11 @@ case class HashAggregateExecTransformer(
val aggRel = if (needsPreProjection) {
getAggRelWithPreProjection(args, originalInputAttributes, input, validation)
} else {
- getAggRelWithoutPreProjection(args, originalInputAttributes, input, validation)
+ if (!GlutenConfig.getConf.isClickHouseBackend) {
+ getAggRelWithoutPreProjection(args, originalInputAttributes, input, validation)
+ } else {
+ getAggRelWithoutPreProjection(args, aggregateResultAttributes, input, validation)
+ }
}
// Will check if post-projection is needed. If yes, a ProjectRel will be added after the
// AggregateRel.
diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala
index dd7f1dc68234..fcc4bffbcde3 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/NativeColumnarToRowExec.scala
@@ -17,27 +17,12 @@
package io.glutenproject.execution
-import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.ConverterUtils
-import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativeColumnarToRowJniWrapper}
-import org.apache.arrow.vector.types.pojo.{Field, Schema}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
-import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
-import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan}
-import org.apache.spark.sql.types._
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-import scala.concurrent.duration._
-
-class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child = child) {
- override def nodeName: String = "NativeColumnarToRow"
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
- override def supportCodegen: Boolean = false
+abstract class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child = child) {
// A flag used to check whether child is wholestage transformer.
// Different backends may have different behaviours according to this flag.
@@ -54,27 +39,7 @@ class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child
true
}
- def buildCheck(): Unit = {
- val schema = child.schema
- for (field <- schema.fields) {
- field.dataType match {
- case d: BooleanType =>
- case d: ByteType =>
- case d: ShortType =>
- case d: IntegerType =>
- case d: LongType =>
- case d: FloatType =>
- case d: DoubleType =>
- case d: StringType =>
- case d: DateType =>
- case d: DecimalType =>
- case d: TimestampType =>
- case d: BinaryType =>
- case _ =>
- throw new UnsupportedOperationException(s"${field.dataType} is not supported in NativeColumnarToRowExec.")
- }
- }
- }
+ def buildCheck(): Unit
override lazy val metrics: Map[String, SQLMetric] = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
@@ -82,117 +47,10 @@ class NativeColumnarToRowExec(child: SparkPlan) extends ColumnarToRowExec(child
"convertTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert")
)
- override def doExecute(): RDD[InternalRow] = {
- val numOutputRows = longMetric("numOutputRows")
- val numInputBatches = longMetric("numInputBatches")
- val convertTime = longMetric("convertTime")
-
- // Fake Arrow format will be returned for WS transformer.
- // TODO: use a Velox layer on the top of the base layer.
- if (GlutenConfig.getConf.isVeloxBackend) {
- if (child.isInstanceOf[WholeStageTransformerExec]) {
- child.asInstanceOf[WholeStageTransformerExec].setFakeOutput()
- }
- }
-
- child.executeColumnar().mapPartitions { batches =>
- // TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast
- val jniWrapper = new NativeColumnarToRowJniWrapper()
- var arrowSchema: Array[Byte] = null
-
- def serializeSchema(fields: Seq[Field]): Array[Byte] = {
- val schema = new Schema(fields.asJava)
- ConverterUtils.getSchemaBytesBuf(schema)
- }
-
- batches.flatMap { batch =>
- numInputBatches += 1
- numOutputRows += batch.numRows()
+ def doExecuteInternal(): RDD[InternalRow]
- if (batch.numRows == 0) {
- logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols")
- Iterator.empty
- } else if (this.output.isEmpty || (batch.numCols() > 0 &&
- !batch.column(0).isInstanceOf[ArrowWritableColumnVector])) {
- // Fallback to ColumnarToRow
- val localOutput = this.output
- numInputBatches += 1
- numOutputRows += batch.numRows()
-
- val toUnsafe = UnsafeProjection.create(localOutput, localOutput)
- batch.rowIterator().asScala.map(toUnsafe)
- } else {
- val bufAddrs = new ListBuffer[Long]()
- val bufSizes = new ListBuffer[Long]()
- val fields = new ListBuffer[Field]()
- (0 until batch.numCols).foreach { idx =>
- val column = batch.column(idx).asInstanceOf[ArrowWritableColumnVector]
- fields += column.getValueVector.getField
- column.getValueVector.getBuffers(false)
- .foreach { buffer =>
- bufAddrs += buffer.memoryAddress()
- try {
- bufSizes += buffer.readableBytes()
- } catch {
- case e: Throwable =>
- // For Velox, the returned format is faked arrow format,
- // and the offset buffer is invalid. Only the buffer address is cared.
- if (GlutenConfig.getConf.isVeloxBackend &&
- child.output(idx).dataType == StringType) {
- // Add a fake value here for String column.
- bufSizes += 1
- } else {
- throw e
- }
- }
- }
- }
-
- if (arrowSchema == null) {
- arrowSchema = serializeSchema(fields)
- }
-
- val beforeConvert = System.nanoTime()
-
- val info = jniWrapper.nativeConvertColumnarToRow(
- arrowSchema, batch.numRows, bufAddrs.toArray, bufSizes.toArray,
- SparkMemoryUtils.contextMemoryPool().getNativeInstanceId, wsChild)
-
- convertTime += NANOSECONDS.toMillis(System.nanoTime() - beforeConvert)
-
- new Iterator[InternalRow] {
- var rowId = 0
- val row = new UnsafeRow(batch.numCols())
- var closed = false
- override def hasNext: Boolean = {
- val result = rowId < batch.numRows()
- if (!result && !closed) {
- jniWrapper.nativeClose(info.instanceID)
- closed = true
- }
- return result
- }
-
- override def next: UnsafeRow = {
- if (rowId >= batch.numRows()) throw new NoSuchElementException
-
- val (offset, length) = (info.offsets(rowId), info.lengths(rowId))
- row.pointTo(null, info.memoryAddress + offset, length.toInt)
- rowId += 1
- row
- }
- }
- }
- }
- }
- }
-
- override def canEqual(other: Any): Boolean = other.isInstanceOf[NativeColumnarToRowExec]
-
- override def equals(other: Any): Boolean = other match {
- case that: NativeColumnarToRowExec =>
- (that canEqual this) && super.equals(that)
- case _ => false
+ override def doExecute(): RDD[InternalRow] = {
+ doExecuteInternal()
}
}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala
index ef7b676f1580..6926e8ed2ea3 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/NativeWholeStageColumnarRDD.scala
@@ -17,49 +17,28 @@
package io.glutenproject.execution
-import io.glutenproject.vectorized.ColumnarFactory
-
import java.io.Serializable
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.ConverterUtils
import io.glutenproject.vectorized._
-import org.apache.arrow.vector.types.pojo.Schema
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.execution.datasources.PartitionedFile
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.util.OASPackageBridge._
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
+import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util._
trait BaseNativeFilePartition extends Partition with InputPartition {
def substraitPlan: Array[Byte]
}
-case class NativeMergeTreePartition(index: Int, engine: String,
- database: String,
- table: String, tablePath: String,
- minParts: Long, maxParts: Long,
- substraitPlan: Array[Byte] = Array.empty[Byte])
- extends BaseNativeFilePartition {
- override def preferredLocations(): Array[String] = {
- Array.empty[String]
- }
-
- def copySubstraitPlan(newSubstraitPlan: Array[Byte]): NativeMergeTreePartition = {
- this.copy(substraitPlan = newSubstraitPlan)
- }
-}
-
case class NativeFilePartition(index: Int, files: Array[PartitionedFile],
substraitPlan: Array[Byte])
extends BaseNativeFilePartition {
@@ -90,7 +69,7 @@ class NativeWholeStageColumnarRDD(
columnarReads: Boolean,
outputAttributes: Seq[Attribute],
jarList: Seq[String],
- dependentKernelIterators: ListBuffer[BatchIterator])
+ dependentKernelIterators: ListBuffer[AbstractBatchIterator])
extends RDD[ColumnarBatch](sc, Nil) {
val numaBindingInfo = GlutenConfig.getConf.numaBindingInfo
val loadNative: Boolean = GlutenConfig.getConf.loadNative
@@ -107,10 +86,7 @@ class NativeWholeStageColumnarRDD(
}
private def castNativePartition(split: Partition): BaseNativeFilePartition = split match {
- case NativeSubstraitPartition(_, p: NativeFilePartition) =>
- p.asInstanceOf[BaseNativeFilePartition]
- case NativeSubstraitPartition(_, p: NativeMergeTreePartition) =>
- p.asInstanceOf[BaseNativeFilePartition]
+ case NativeSubstraitPartition(_, p: BaseNativeFilePartition) => p
case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split")
}
@@ -119,73 +95,8 @@ class NativeWholeStageColumnarRDD(
val inputPartition = castNativePartition(split)
- var inputSchema : Schema = null
- var outputSchema : Schema = null
- var resIter : BatchIterator = null
- if (loadNative) {
- // TODO: 'jarList' is kept for codegen
- val transKernel = new ExpressionEvaluator(jarList.toList.asJava)
- val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]()
- outputSchema = ConverterUtils.toArrowSchema(outputAttributes)
- resIter = transKernel.createKernelWithBatchIterator(
- inputPartition.substraitPlan, inBatchIters)
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ => resIter.close() }
- }
- val iter = new Iterator[Any] {
- private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics
-
- override def hasNext: Boolean = {
- if (loadNative) {
- resIter.hasNext
- } else {
- false
- }
- }
-
- def nextArrowColumnarBatch(): Any = {
- val rb = resIter.next()
- if (rb == null) {
- val resultStructType = ArrowUtils.fromArrowSchema(outputSchema)
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
- return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
- }
- val outputNumRows = rb.getLength
- val output = ConverterUtils.fromArrowRecordBatch(outputSchema, rb)
- ConverterUtils.releaseArrowRecordBatch(rb)
- val cb = new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows)
- val bytes: Long = cb match {
- case batch: ColumnarBatch =>
- (0 until batch.numCols()).map { i =>
- val vector = Option(batch.column(i))
- vector.map {
- case av: ArrowWritableColumnVector =>
- av.getValueVector.getBufferSize.toLong
- case _ => 0L
- }.sum
- }.sum
- case _ => 0L
- }
- inputMetrics.bridgeIncBytesRead(bytes)
- cb
- }
-
- override def next(): Any = {
- if (!hasNext) {
- throw new java.util.NoSuchElementException("End of stream")
- }
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- nextArrowColumnarBatch()
- } else {
- resIter.chNext()
- }
-
- }
- }
-
- // TODO: SPARK-25083 remove the type erasure hack in data source scan
- new InterruptibleIterator(context,
- ColumnarFactory.createClosableIterator(iter.asInstanceOf[Iterator[ColumnarBatch]]))
+ BackendsApiManager.getIteratorApiInstance.genFirstStageIterator(inputPartition, loadNative,
+ outputAttributes, context, jarList)
}
override def getPreferredLocations(split: Partition): Seq[String] = {
diff --git a/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala b/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala
index ba787ea15613..0878f3802a20 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/NativeWholestageRowRDD.scala
@@ -17,8 +17,6 @@
package io.glutenproject.execution
-import java.io._
-
import io.glutenproject.GlutenConfig
import io.glutenproject.row.RowIterator
import io.glutenproject.vectorized.ExpressionEvaluator
@@ -50,8 +48,7 @@ class NativeWholestageRowRDD(
}
private def castNativePartition(split: Partition): BaseNativeFilePartition = split match {
- case NativeSubstraitPartition(_, p: NativeFilePartition) => p
- case NativeSubstraitPartition(_, m: NativeMergeTreePartition) => m
+ case NativeSubstraitPartition(_, p: BaseNativeFilePartition) => p
case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split")
}
@@ -63,7 +60,7 @@ class NativeWholestageRowRDD(
var resIter : RowIterator = null
if (loadNative) {
val transKernel = new ExpressionEvaluator()
- val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]()
+ val inBatchIters = new java.util.ArrayList[AbstractColumnarNativeIterator]()
var startTime = System.nanoTime()
resIter = transKernel.createKernelWithRowIterator(inputPartition.substraitPlan, inBatchIters)
logWarning(s"===========create ${System.nanoTime() - startTime}")
diff --git a/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala b/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala
index 03ab53cdcb93..350bdfe919ed 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/RowToArrowColumnarExec.scala
@@ -19,211 +19,17 @@ package io.glutenproject.execution
import java.util.concurrent.TimeUnit._
-import io.glutenproject.vectorized._
-
import org.apache.spark.broadcast
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.expressions.SortOrder
-import org.apache.spark.sql.catalyst.expressions.SpecializedGetters
+import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils.UnsafeItr
-import org.apache.spark.sql.execution.metric.SQLMetric
-import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector
-import org.apache.spark.sql.execution.vectorized.WritableColumnVector
-import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
-class RowToColumnConverter(schema: StructType) extends Serializable {
- private val converters = schema.fields.map {
- f => RowToColumnConverter.getConverterForType(f.dataType, f.nullable)
- }
-
- final def convert(row: InternalRow, vectors: Array[WritableColumnVector]): Unit = {
- var idx = 0
- while (idx < row.numFields) {
- converters(idx).append(row, idx, vectors(idx))
- idx += 1
- }
- }
-}
-
-object RowToColumnConverter {
- private abstract class TypeConverter extends Serializable {
- def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit
- }
-
- private final case class BasicNullableTypeConverter(base: TypeConverter) extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- if (row.isNullAt(column)) {
- cv.appendNull
- } else {
- base.append(row, column, cv)
- }
- }
- }
-
- private final case class StructNullableTypeConverter(base: TypeConverter) extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- if (row.isNullAt(column)) {
- cv.appendStruct(true)
- } else {
- base.append(row, column, cv)
- }
- }
- }
-
- private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = {
- val core = dataType match {
- case BooleanType => BooleanConverter
- case ByteType => ByteConverter
- case ShortType => ShortConverter
- case IntegerType | DateType => IntConverter
- case FloatType => FloatConverter
- case LongType | TimestampType => LongConverter
- case DoubleType => DoubleConverter
- case StringType => StringConverter
- case BinaryType => BinaryConverter
- case CalendarIntervalType => CalendarConverter
- case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable))
- case st: StructType => new StructConverter(st.fields.map(
- (f) => getConverterForType(f.dataType, f.nullable)))
- case dt: DecimalType => new DecimalConverter(dt)
- case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable),
- getConverterForType(mt.valueType, nullable))
- case unknown => throw new UnsupportedOperationException(
- s"Type $unknown not supported")
- }
-
- if (nullable) {
- dataType match {
- case CalendarIntervalType => new StructNullableTypeConverter(core)
- case st: StructType => new StructNullableTypeConverter(core)
- case _ => new BasicNullableTypeConverter(core)
- }
- } else {
- core
- }
- }
-
- private object BooleanConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendBoolean(row.getBoolean(column))
- }
-
- private object ByteConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendByte(row.getByte(column))
- }
-
- private object ShortConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendShort(row.getShort(column))
- }
-
- private object IntConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendInt(row.getInt(column))
- }
-
- private object FloatConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendFloat(row.getFloat(column))
- }
-
- private object LongConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendLong(row.getLong(column))
- }
-
- private object DoubleConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit =
- cv.appendDouble(row.getDouble(column))
- }
-
- private object StringConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val data = row.getUTF8String(column).getBytes
- cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length)
- }
- }
-
- private object BinaryConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val data = row.getBinary(column)
- cv.asInstanceOf[ArrowWritableColumnVector].appendString(data, 0, data.length)
- }
- }
-
- private object CalendarConverter extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val c = row.getInterval(column)
- cv.appendStruct(false)
- cv.getChild(0).appendInt(c.months)
- cv.getChild(1).appendInt(c.days)
- cv.getChild(2).appendLong(c.microseconds)
- }
- }
-
- private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val values = row.getArray(column)
- val numElements = values.numElements()
- cv.appendArray(numElements)
- val arrData = cv.arrayData()
- for (i <- 0 until numElements) {
- childConverter.append(values, i, arrData)
- }
- }
- }
-
- private case class StructConverter(childConverters: Array[TypeConverter]) extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- cv.appendStruct(false)
- val data = row.getStruct(column, childConverters.length)
- for (i <- 0 until childConverters.length) {
- childConverters(i).append(data, i, cv.getChild(i))
- }
- }
- }
-
- private case class DecimalConverter(dt: DecimalType) extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val d = row.getDecimal(column, dt.precision, dt.scale)
- if (dt.precision <= Decimal.MAX_INT_DIGITS) {
- cv.appendInt(d.toUnscaledLong.toInt)
- } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) {
- cv.appendLong(d.toUnscaledLong)
- } else {
- val value = d.toJavaBigDecimal
- cv.asInstanceOf[ArrowWritableColumnVector].appendDecimal(value)
- }
- }
- }
-
- private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter)
- extends TypeConverter {
- override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = {
- val m = row.getMap(column)
- val keys = cv.getChild(0)
- val values = cv.getChild(1)
- val numElements = m.numElements()
- cv.appendArray(numElements)
-
- val srcKeys = m.keyArray()
- val srcValues = m.valueArray()
-
- for (i <- 0 until numElements) {
- keyConverter.append(srcKeys, i, keys)
- valueConverter.append(srcValues, i, values)
- }
- }
- }
-}
-
/**
* Provides a common executor to translate an [[RDD]] of [[InternalRow]] into an [[RDD]] of
* [[ColumnarBatch]]. This is inserted whenever such a transition is determined to be needed.
@@ -241,7 +47,7 @@ object RowToColumnConverter {
* populate with [[RowToColumnConverter]], but the performance requirements are different and it
* would only be to reduce code.
*/
-case class RowToArrowColumnarExec(child: SparkPlan) extends UnaryExecNode {
+abstract class RowToArrowColumnarExec(child: SparkPlan) extends RowToColumnarExec(child = child) {
override def output: Seq[Attribute] = child.output
override def outputPartitioning: Partitioning = child.outputPartitioning
@@ -264,50 +70,9 @@ case class RowToArrowColumnarExec(child: SparkPlan) extends UnaryExecNode {
"processTime" -> SQLMetrics.createTimingMetric(sparkContext, "totaltime_rowtoarrowcolumnar")
)
- override def doExecuteColumnar(): RDD[ColumnarBatch] = {
- val numInputRows = longMetric("numInputRows")
- val numOutputBatches = longMetric("numOutputBatches")
- val processTime = longMetric("processTime")
- // Instead of creating a new config we are reusing columnBatchSize. In the future if we do
- // combine with some of the Arrow conversion tools we will need to unify some of the configs.
- val numRows = conf.columnBatchSize
- // This avoids calling `schema` in the RDD closure, so that we don't need to include the entire
- // plan (this) in the closure.
- val localSchema = this.schema
- child.execute().mapPartitions { rowIterator =>
- if (rowIterator.hasNext) {
- val res = new Iterator[ColumnarBatch] {
- private val converters = new RowToColumnConverter(localSchema)
- private var last_cb: ColumnarBatch = null
- private var elapse: Long = 0
-
- override def hasNext: Boolean = {
- rowIterator.hasNext
- }
+ def doExecuteColumnarInternal(): RDD[ColumnarBatch]
- override def next(): ColumnarBatch = {
- val vectors: Seq[WritableColumnVector] =
- ArrowWritableColumnVector.allocateColumns(numRows, schema)
- var rowCount = 0
- while (rowCount < numRows && rowIterator.hasNext) {
- val row = rowIterator.next()
- val start = System.nanoTime()
- converters.convert(row, vectors.toArray)
- elapse += System.nanoTime() - start
- rowCount += 1
- }
- vectors.foreach(v => v.asInstanceOf[ArrowWritableColumnVector].setValueCount(rowCount))
- processTime.set(NANOSECONDS.toMillis(elapse))
- numInputRows += rowCount
- numOutputBatches += 1
- last_cb = new ColumnarBatch(vectors.toArray, rowCount)
- last_cb
- }
- }
- new UnsafeItr(res)
- } else {
- Iterator.empty
- }
- }
+ override def doExecuteColumnar(): RDD[ColumnarBatch] = {
+ doExecuteColumnarInternal()
}
}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala
index 0f3b29216e3d..09f6014829a6 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/ShuffledHashJoinExecTransformer.scala
@@ -17,43 +17,19 @@
package io.glutenproject.execution
-import java.util.concurrent.TimeUnit._
-
-import io.glutenproject.vectorized._
-import io.glutenproject.GlutenConfig
-import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
-import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils, Utils}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan}
+import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.metric.SQLMetrics
-import scala.collection.JavaConverters._
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-
-import scala.collection.mutable.ListBuffer
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.arrow.vector.types.pojo.Field
-import org.apache.arrow.vector.types.pojo.Schema
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.gandiva.evaluator._
-import org.apache.arrow.memory.ArrowBuf
-import com.google.common.collect.Lists
-import io.glutenproject.expression._
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized.ExpressionEvaluator
-import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide}
import org.apache.spark.sql.execution.joins.{BaseJoinExec, HashJoin, ShuffledJoin}
-import org.apache.spark.sql.execution.joins.HashedRelationInfo
/**
* Performs a hash join of two child relations by first shuffling the data using the join keys.
diff --git a/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala
index 57c98021075c..9bdc87527566 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala
@@ -17,35 +17,14 @@
package io.glutenproject.execution
-import io.glutenproject.GlutenConfig
-import io.glutenproject.expression._
-import io.glutenproject.vectorized._
-import com.google.common.collect.Lists
-import java.util.concurrent.TimeUnit._
-
import io.glutenproject.substrait.SubstraitContext
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.arrow.vector.types.pojo.Field
-import org.apache.arrow.vector.types.pojo.Schema
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.gandiva.evaluator._
-import org.apache.spark.{SparkContext, SparkEnv, TaskContext}
-import org.apache.spark.executor.TaskMetrics
import org.apache.spark.sql.execution._
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.BoundReference
-import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.rdd.RDD
-import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils, Utils}
-import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-import org.apache.spark.sql.types.DecimalType
-
-import scala.util.control.Breaks.{break, breakable}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.vectorized.ColumnarBatch
case class SortExecTransformer(
sortOrder: Seq[SortOrder],
diff --git a/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala
index 8e82d71408a5..be6db6c1c531 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala
@@ -17,43 +17,16 @@
package io.glutenproject.execution
-import java.util.concurrent.TimeUnit._
+import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized._
-import io.glutenproject.GlutenConfig
-import org.apache.spark.TaskContext
import org.apache.spark.rdd.RDD
-import org.apache.spark.util.{UserAddedJarUtils, Utils}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.metric.SQLMetrics
-
-import scala.collection.JavaConverters._
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.expressions.BoundReference
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-
-import scala.collection.mutable.ListBuffer
-import org.apache.arrow.vector.ipc.message.ArrowFieldNode
-import org.apache.arrow.vector.ipc.message.ArrowRecordBatch
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.arrow.vector.types.pojo.Field
-import org.apache.arrow.vector.types.pojo.Schema
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.gandiva.evaluator._
-import org.apache.arrow.memory.ArrowBuf
-import com.google.common.collect.Lists
-import io.glutenproject.expression._
-import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized.ExpressionEvaluator
-import org.apache.spark.sql.execution.joins._
-import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide}
-import org.apache.spark.sql.types.DecimalType
+import org.apache.spark.sql.vectorized.ColumnarBatch
/**
* Performs a hash join of two child relations by first shuffling the data using the join keys.
diff --git a/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala b/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala
index 7fc1b332618f..535d0f672bf1 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/WholeStageTransformerExec.scala
@@ -21,27 +21,22 @@ import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import com.google.common.collect.Lists
-import io.glutenproject.vectorized.ColumnarFactory
import io.glutenproject.GlutenConfig
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.expression._
import io.glutenproject.substrait.plan.{PlanBuilder, PlanNode}
-import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder, RelNode}
+import io.glutenproject.substrait.rel.RelNode
import io.glutenproject.substrait.SubstraitContext
import io.glutenproject.vectorized._
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.vector.types.pojo.{ArrowType, Field}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder}
import org.apache.spark.sql.catalyst.plans.physical.Partitioning
import org.apache.spark.sql.execution._
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
-import org.apache.spark.sql.execution.datasources.FilePartition
import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
-import org.apache.spark.util.{ExecutorManager, UserAddedJarUtils}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.util.UserAddedJarUtils
case class TransformContext(inputAttributes: Seq[Attribute],
outputAttributes: Seq[Attribute], root: RelNode)
@@ -109,6 +104,7 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator
override def otherCopyArgs: Seq[AnyRef] = Seq(transformStageId.asInstanceOf[Integer])
+
override def generateTreeString(
depth: Int,
lastChildren: Seq[Boolean],
@@ -133,6 +129,7 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
}
override def nodeName: String = s"WholeStageCodegenTransformer (${transformStageId})"
+
def uploadAndListJars(signature: String): Seq[String] =
if (signature != "") {
if (sparkContext.listJars.filter(path => path.contains(s"${signature}.jar")).isEmpty) {
@@ -166,7 +163,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
}
}
for (attr <- childCtx.outputAttributes) {
- outNames.add(attr.name)
+ // ============================================
+ // outNames.add(attr.name)
+ outNames.add(ConverterUtils.getShortAttributeName(attr) + "#" + attr.exprId.id)
}
val planNode = PlanBuilder.makePlan(
substraitContext, Lists.newArrayList(childCtx.root), outNames)
@@ -188,7 +187,8 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
override def updateMetrics(out_num_rows: Long, process_time: Long): Unit = {}
var metricsUpdated: Boolean = false
- def updateMetrics(nativeIterator: BatchIterator): Unit = {
+
+ def updateMetrics(nativeIterator: AbstractBatchIterator): Unit = {
if (metricsUpdated) return
try {
val metrics = nativeIterator.getMetrics
@@ -212,47 +212,6 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
}
}
- def prepareRelationFunction(
- keyAttributes: Seq[Attribute],
- outputAttributes: Seq[Attribute]): TreeNode = {
- val outputFieldList: List[Field] = outputAttributes.toList.map(attr => {
- Field
- .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}",
- CodeGeneration.getResultType(attr.dataType))
- })
-
- val keyFieldList: List[Field] = keyAttributes.toList.map(attr => {
- val field = Field
- .nullable(s"${attr.name.toUpperCase()}#${attr.exprId.id}",
- CodeGeneration.getResultType(attr.dataType))
- if (outputFieldList.indexOf(field) == -1) {
- throw new UnsupportedOperationException(s"CachedRelation not found" +
- s"${attr.name.toUpperCase()}#${attr.exprId.id} in ${outputAttributes}")
- }
- field
- })
-
- val key_args_node = TreeBuilder.makeFunction(
- "key_field",
- keyFieldList
- .map(field => {
- TreeBuilder.makeField(field)
- })
- .asJava,
- new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ )
-
- val cachedRelationFuncName = "CachedRelation"
- val cached_relation_func = TreeBuilder.makeFunction(
- cachedRelationFuncName,
- Lists.newArrayList(key_args_node),
- new ArrowType.Int(32, true) /*dummy ret type, won't be used*/ )
-
- TreeBuilder.makeFunction(
- "standalone",
- Lists.newArrayList(cached_relation_func),
- new ArrowType.Int(32, true))
- }
-
/**
* Return built cpp library's signature
*/
@@ -300,37 +259,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
val wsCxt = doWholestageTransform()
val startTime = System.nanoTime()
- val substraitPlanPartition = fileScan.getPartitions.map {
- case p: NativeMergeTreePartition =>
- val extensionTableNode =
- ExtensionTableBuilder.makeExtensionTable(p.minParts,
- p.maxParts, p.database, p.table, p.tablePath)
- wsCxt.substraitContext.setExtensionTableNode(extensionTableNode)
- // logWarning(s"The substrait plan for partition " +
- // s"${p.index}:\n${wsCxt.root.toProtobuf.toString}")
- p.copySubstraitPlan(wsCxt.root.toProtobuf.toByteArray)
- case FilePartition(index, files) =>
- val paths = new java.util.ArrayList[String]()
- val starts = new java.util.ArrayList[java.lang.Long]()
- val lengths = new java.util.ArrayList[java.lang.Long]()
- files.foreach { f =>
- paths.add(f.filePath)
- starts.add(new java.lang.Long(f.start))
- lengths.add(new java.lang.Long(f.length))
- }
- val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths)
- wsCxt.substraitContext.setLocalFilesNode(localFilesNode)
- val substraitPlan = wsCxt.root.toProtobuf
- /*
- val out = new DataOutputStream(new FileOutputStream("/tmp/SubStraitTest-Q6.dat",
- false));
- out.write(substraitPlan.toByteArray());
- out.flush();
- */
- // logWarning(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}")
- NativeFilePartition(index, files, substraitPlan.toByteArray)
- case p => p
- }
+ val substraitPlanPartition = fileScan.getPartitions.map(p =>
+ BackendsApiManager.getIteratorApiInstance.genNativeFilePartition(p, wsCxt)
+ )
logWarning(
s"Generated substrait plan tooks: ${(System.nanoTime() - startTime) / 1000000} ms")
@@ -348,17 +279,15 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
val numOutputBatches = child.longMetric("numOutputBatches")
val pipelineTime = longMetric("pipelineTime")
- var build_elapse: Long = 0
- var eval_elapse: Long = 0
// we should zip all dependent RDDs to current main RDD
// TODO: Does it still need these parameters?
val buildPlans = getBuildPlans
val streamedSortPlan = getStreamedLeafPlan
val dependentKernels: ListBuffer[ExpressionEvaluator] = ListBuffer()
- val dependentKernelIterators: ListBuffer[BatchIterator] = ListBuffer()
+ val dependentKernelIterators: ListBuffer[AbstractBatchIterator] = ListBuffer()
val buildRelationBatchHolder: ListBuffer[ColumnarBatch] = ListBuffer()
- val serializableObjectHolder: ListBuffer[SerializableObject] = ListBuffer()
- val relationHolder: ListBuffer[ColumnarHashedRelation] = ListBuffer()
+ // val serializableObjectHolder: ListBuffer[SerializableObject] = ListBuffer()
+ // val relationHolder: ListBuffer[ColumnarHashedRelation] = ListBuffer()
// check if BatchScan exists
val current_op = checkBatchScanExecTransformerChild()
@@ -379,33 +308,9 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
val wsCxt = doWholestageTransform()
val startTime = System.nanoTime()
- val substraitPlanPartition = fileScan.getPartitions.map {
- case p: NativeMergeTreePartition =>
- val extensionTableNode =
- ExtensionTableBuilder.makeExtensionTable(p.minParts,
- p.maxParts, p.database, p.table, p.tablePath)
- wsCxt.substraitContext.setExtensionTableNode(extensionTableNode)
-// logWarning(s"The substrait plan for partition " +
-// s"${p.index}:\n${wsCxt.root.toProtobuf.toString}")
- p.copySubstraitPlan(wsCxt.root.toProtobuf.toByteArray)
- case FilePartition(index, files) =>
- val paths = new java.util.ArrayList[String]()
- val starts = new java.util.ArrayList[java.lang.Long]()
- val lengths = new java.util.ArrayList[java.lang.Long]()
- files.foreach { f =>
- paths.add(f.filePath)
- starts.add(new java.lang.Long(f.start))
- lengths.add(new java.lang.Long(f.length))
- }
-
- val localFilesNode = LocalFilesBuilder.makeLocalFiles(index, paths, starts, lengths)
- wsCxt.substraitContext.setLocalFilesNode(localFilesNode)
- val substraitPlan = wsCxt.root.toProtobuf
-
- logInfo(s"The substrait plan for partition ${index}:\n${substraitPlan.toString}")
- NativeFilePartition(index, files, substraitPlan.toByteArray)
- case p => p
- }
+ val substraitPlanPartition = fileScan.getPartitions.map(p =>
+ BackendsApiManager.getIteratorApiInstance.genNativeFilePartition(p, wsCxt)
+ )
logWarning(
s"Generating the Substrait plan took: ${(System.nanoTime() - startTime) / 1000000} ms.")
@@ -426,124 +331,15 @@ case class WholeStageTransformerExec(child: SparkPlan)(val transformStageId: Int
val resCtx = doWholestageTransform()
val outputAttributes = resCtx.outputAttributes
val rootNode = resCtx.root
-
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- curRDD.mapPartitions { iter =>
- ExecutorManager.tryTaskSet(numaBindingInfo)
- GlutenConfig.getConf
- val execTempDir = GlutenConfig.getTempFile
- val jarList = listJars.map(jarUrl => {
- logWarning(s"Get Codegened library Jar ${jarUrl}")
- UserAddedJarUtils.fetchJarFromSpark(
- jarUrl,
- execTempDir,
- s"spark-columnar-plugin-codegen-precompile-${signature}.jar",
- sparkConf)
- s"${execTempDir}/spark-columnar-plugin-codegen-precompile-${signature}.jar"
- })
-
- val transKernel = new ExpressionEvaluator(jarList.toList.asJava)
- val inBatchIter = new ColumnarNativeIterator(iter.asJava)
- val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]()
- inBatchIters.add(inBatchIter)
- // we need to complete dependency RDD's firstly
- val beforeBuild = System.nanoTime()
- val outputSchema = ConverterUtils.toArrowSchema(outputAttributes)
- val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters)
- build_elapse += System.nanoTime() - beforeBuild
- val resultStructType = ArrowUtils.fromArrowSchema(outputSchema)
- val resIter = streamedSortPlan match {
- case t: TransformSupport =>
- new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = {
- val res = nativeIterator.hasNext
- // if (res == false) updateMetrics(nativeIterator)
- res
- }
-
- override def next(): ColumnarBatch = {
- val beforeEval = System.nanoTime()
- val output_rb = nativeIterator.next
- if (output_rb == null) {
- eval_elapse += System.nanoTime() - beforeEval
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
- return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
- }
- val outputNumRows = output_rb.getLength
- val outSchema = ConverterUtils.toArrowSchema(resCtx.outputAttributes)
- val output = ConverterUtils.fromArrowRecordBatch(outSchema, output_rb)
- ConverterUtils.releaseArrowRecordBatch(output_rb)
- eval_elapse += System.nanoTime() - beforeEval
- new ColumnarBatch(output.map(v => v.asInstanceOf[ColumnVector]), outputNumRows)
- }
- }
- case _ =>
- throw new UnsupportedOperationException(
- s"streamedSortPlan should support transformation")
- }
- var closed = false
-
- def close = {
- closed = true
- pipelineTime += (eval_elapse + build_elapse) / 1000000
- buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative
- dependentKernels.foreach(_.close)
- dependentKernelIterators.foreach(_.close)
- nativeIterator.close()
- relationHolder.clear()
- }
-
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => {
- close
- })
- ColumnarFactory.createClosableIterator(resIter)
- }
- } else {
- curRDD.mapPartitions { iter =>
- GlutenConfig.getConf
- val transKernel = new ExpressionEvaluator()
- val inBatchIter = new CHColumnarNativeIterator(iter.asJava)
- val inBatchIters = new java.util.ArrayList[ColumnarNativeIterator]()
- inBatchIters.add(inBatchIter)
- // we need to complete dependency RDD's firstly
- val beforeBuild = System.nanoTime()
- val nativeIterator = transKernel.createKernelWithBatchIterator(rootNode, inBatchIters)
- build_elapse += System.nanoTime() - beforeBuild
- val resIter = streamedSortPlan match {
- case t: TransformSupport =>
- new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = {
- val res = nativeIterator.hasNext
- res
- }
-
- override def next(): ColumnarBatch = {
- val beforeEval = System.nanoTime()
- nativeIterator.chNext()
- }
- }
- case _ =>
- throw new UnsupportedOperationException(
- s"streamedSortPlan should support transformation")
- }
- var closed = false
-
- def close = {
- closed = true
- pipelineTime += (eval_elapse + build_elapse) / 1000000
- buildRelationBatchHolder.foreach(_.close) // fixing: ref cnt goes nagative
- dependentKernels.foreach(_.close)
- dependentKernelIterators.foreach(_.close)
- nativeIterator.close()
- relationHolder.clear()
- }
-
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit](_ => {
- close
- })
- ColumnarFactory.createClosableIterator(resIter)
- }
+ // logInfo(s"The substrait plan for final agg " +
+ // s"\n${rootNode.toProtobuf.toString}")
+
+ curRDD.mapPartitions { iter =>
+ BackendsApiManager.getIteratorApiInstance.genFinalStageIterator(iter, numaBindingInfo,
+ listJars, signature, sparkConf, outputAttributes,
+ rootNode, streamedSortPlan,
+ pipelineTime, buildRelationBatchHolder,
+ dependentKernels, dependentKernelIterators)
}
}
}
diff --git a/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala b/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala
index cb2c79474852..b9f88e6834da 100644
--- a/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala
@@ -17,38 +17,16 @@
package io.glutenproject.execution
-import java.util.concurrent.TimeUnit
-
-import com.google.flatbuffers.FlatBufferBuilder
-import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.{CodeGeneration, ConverterUtils}
import io.glutenproject.substrait.SubstraitContext
-import io.glutenproject.vectorized.{ArrowWritableColumnVector, CloseableColumnBatchIterator, ExpressionEvaluator}
-import org.apache.arrow.gandiva.expression.TreeBuilder
-import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID
-import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}
-import org.apache.spark.internal.Logging
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, Cast, Descending, Expression, Literal, MakeDecimal, NamedExpression, PredicateHelper, Rank, SortOrder, UnscaledValue, WindowExpression, WindowFunction, WindowSpecDefinition}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder}
import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning}
-import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.{SortExec, SparkPlan}
+import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.execution.window.WindowExecBase
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DataType, DateType, DecimalType, DoubleType, IntegerType, LongType, StringType, TimestampType}
-import org.apache.spark.sql.util.ArrowUtils
import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.ExecutorManager
-
-import scala.collection.JavaConverters._
-import scala.collection.immutable.Stream.Empty
-import scala.collection.mutable.ListBuffer
-import scala.util.Random
-import util.control.Breaks._
case class WindowExecTransformer(windowExpression: Seq[NamedExpression],
partitionSpec: Seq[Expression],
@@ -103,7 +81,7 @@ case class WindowExecTransformer(windowExpression: Seq[NamedExpression],
override def hashCode(): Int = System.identityHashCode(this)
- private object NoneType {
+ /* private object NoneType {
val NONE_TYPE = new NoneType
}
@@ -125,7 +103,7 @@ case class WindowExecTransformer(windowExpression: Seq[NamedExpression],
}
override def isComplex: Boolean = false
- }
+ } */
override protected def doExecute(): RDD[InternalRow] = {
throw new UnsupportedOperationException()
diff --git a/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala
index 7d8140bb1516..750c20ca5d88 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/BinaryExpressionTransformer.scala
@@ -20,9 +20,7 @@ package io.glutenproject.expression
import com.google.common.collect.Lists
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-import scala.collection.mutable.ListBuffer
import io.glutenproject.expression.DateTimeExpressionsTransformer.{DateDiffTransformer, UnixTimestampTransformer}
import io.glutenproject.substrait.`type`.TypeBuilder
import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode}
diff --git a/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala
index 9be7e3883279..c5d1000bcd26 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/BoundReferenceTransformer.scala
@@ -17,10 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode}
-import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.internal.Logging
import org.apache.spark.sql.types._
class BoundReferenceTransformer(ordinal: Int, dataType: DataType, nullable: Boolean)
diff --git a/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala
index 2adf3c47021b..236fe89bf311 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/CaseWhenOperatorTransformer.scala
@@ -17,14 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
/**
* A version of substring that supports columnar processing for utf8.
diff --git a/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala
index ed4a705c7582..10bc9dd4d384 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/CoalesceOperatorTransformer.scala
@@ -17,14 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
/**
* An expression that is evaluated to the first non-null input.
diff --git a/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala
index bfd9d061e5a7..c002fafdc2fc 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/ConcatOperatorTransformer.scala
@@ -16,14 +16,9 @@
*/
package io.glutenproject.expression
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
class ConcatTransformer(exps: Seq[Expression], original: Expression)
extends Concat(exps: Seq[Expression])
diff --git a/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala b/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala
index e09b36f37555..2c5225871621 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/ConverterUtils.scala
@@ -17,287 +17,16 @@
package io.glutenproject.expression
-import java.io._
-import java.nio.channels.Channels
-
-import scala.collection.JavaConverters._
-
-import com.google.common.collect.Lists
import io.glutenproject.substrait.`type`._
-import io.glutenproject.vectorized.ArrowWritableColumnVector
-import io.netty.buffer.{ByteBufAllocator, ByteBufOutputStream}
-import org.apache.arrow.flatbuf.MessageHeader
-import org.apache.arrow.gandiva.evaluator._
-import org.apache.arrow.gandiva.exceptions.GandivaException
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.gandiva.ipc.GandivaTypes
-import org.apache.arrow.gandiva.ipc.GandivaTypes.ExpressionList
-import org.apache.arrow.memory.BufferAllocator
-import org.apache.arrow.vector._
-import org.apache.arrow.vector.ipc.{ReadChannel, WriteChannel}
-import org.apache.arrow.vector.ipc.message._
-import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}
-import org.apache.arrow.vector.types.TimeUnit
-import org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.optimizer._
-import org.apache.spark.sql.catalyst.util.DateTimeConstants
-import org.apache.spark.sql.execution.datasources.v2.arrow.{SparkSchemaUtils, SparkVectorUtils}
import org.apache.spark.sql.types._
-import org.apache.spark.sql.util.ArrowUtils
-import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
+import org.apache.spark.sql.vectorized.ColumnarBatch
object ConverterUtils extends Logging {
- def calcuateEstimatedSize(columnarBatch: ColumnarBatch): Long = {
- SparkVectorUtils.estimateSize(columnarBatch)
- }
-
- def createArrowRecordBatch(columnarBatch: ColumnarBatch): ArrowRecordBatch = {
- SparkVectorUtils.toArrowRecordBatch(columnarBatch)
- }
-
- def createArrowRecordBatch(numRowsInBatch: Int, cols: List[ValueVector]): ArrowRecordBatch = {
- SparkVectorUtils.toArrowRecordBatch(numRowsInBatch, cols)
- }
-
- def convertToNetty(iter: Array[ColumnarBatch], out: OutputStream): Unit = {
- val channel = new WriteChannel(Channels.newChannel(out))
- var schema: Schema = null
- val option = new IpcOption
-
- iter.foreach { columnarBatch =>
- val vectors = (0 until columnarBatch.numCols)
- .map(i => columnarBatch.column(i).asInstanceOf[ArrowWritableColumnVector])
- .toList
- try {
- if (schema == null) {
- schema = new Schema(vectors.map(_.getValueVector().getField).asJava)
- MessageSerializer.serialize(channel, schema, option)
- }
- val batch = ConverterUtils
- .createArrowRecordBatch(columnarBatch.numRows, vectors.map(_.getValueVector))
- try {
- MessageSerializer.serialize(channel, batch, option)
- } finally {
- batch.close()
- }
- } catch {
- case e =>
- System.err.println(s"conversion failed")
- e.printStackTrace()
- throw e
- }
- }
- }
-
- def convertToNetty(iter: Array[ColumnarBatch]): Array[Byte] = {
- val innerBuf = ByteBufAllocator.DEFAULT.buffer()
- val outStream = new ByteBufOutputStream(innerBuf)
- convertToNetty(iter, outStream)
- val bytes = new Array[Byte](innerBuf.readableBytes);
- innerBuf.getBytes(innerBuf.readerIndex, bytes);
- innerBuf.release()
- outStream.close()
- bytes
- }
-
- def convertFromNetty(
- attributes: Seq[Attribute],
- input: InputStream): Iterator[ColumnarBatch] = {
- new Iterator[ColumnarBatch] {
- val allocator = ArrowWritableColumnVector.getOffRecordAllocator
- var messageReader =
- new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
- var schema: Schema = null
- var result: MessageResult = null
-
- override def hasNext: Boolean =
- if (input.available > 0) {
- return true
- } else {
- messageReader.close
- return false
- }
- override def next(): ColumnarBatch = {
- if (input.available == 0) {
- if (attributes == null) {
- return null
- }
- val resultStructType = StructType(
- attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
- return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
- }
- try {
- if (schema == null) {
- result = messageReader.readNext();
- if (result == null) {
- throw new IOException("Unexpected end of input. Missing schema.");
- }
- if (result.getMessage().headerType() != MessageHeader.Schema) {
- throw new IOException(
- "Expected schema but header was " + result.getMessage().headerType());
- }
- schema = MessageSerializer.deserializeSchema(result.getMessage());
- }
- result = messageReader.readNext();
- if (result.getMessage().headerType() != MessageHeader.RecordBatch) {
- throw new IOException(
- "Expected recordbatch but header was " + result.getMessage().headerType());
- }
- var bodyBuffer = result.getBodyBuffer();
-
- // For zero-length batches, need an empty buffer to deserialize the batch
- if (bodyBuffer == null) {
- bodyBuffer = allocator.getEmpty();
- }
-
- val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer);
- val vectors = fromArrowRecordBatch(schema, batch, allocator)
- val length = batch.getLength
- batch.close
- new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length)
- } catch {
- case e: Throwable =>
- messageReader.close
- throw e
- }
- }
- }
- }
-
- def convertFromNetty(
- attributes: Seq[Attribute],
- data: Array[Array[Byte]],
- columnIndices: Array[Int] = null): Iterator[ColumnarBatch] = {
- if (data.size == 0) {
- return new Iterator[ColumnarBatch] {
- override def hasNext: Boolean = false
- override def next(): ColumnarBatch = {
- val resultStructType = if (columnIndices == null) {
- StructType(
- attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
- } else {
- StructType(
- columnIndices
- .map(i => attributes(i))
- .map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
- }
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
- return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
- }
- }
- }
- new Iterator[ColumnarBatch] {
- var array_id = 0
- val allocator = ArrowWritableColumnVector.getOffRecordAllocator
- var input = new ByteArrayInputStream(data(array_id))
- var messageReader =
- new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
- var schema: Schema = null
- var result: MessageResult = null
-
- override def hasNext: Boolean =
- if (array_id < (data.size - 1) || input.available > 0) {
- return true
- } else {
- messageReader.close
- return false
- }
- override def next(): ColumnarBatch = {
- if (input.available == 0) {
- messageReader.close
- array_id += 1
- input = new ByteArrayInputStream(data(array_id))
- messageReader =
- new MessageChannelReader(new ReadChannel(Channels.newChannel(input)), allocator)
- }
- if (input.available == 0) {
- val resultStructType = StructType(
- attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))
- val resultColumnVectors =
- ArrowWritableColumnVector.allocateColumns(0, resultStructType).toArray
- return new ColumnarBatch(resultColumnVectors.map(_.asInstanceOf[ColumnVector]), 0)
- }
- try {
- if (schema == null) {
- result = messageReader.readNext();
-
- if (result == null) {
- throw new IOException("Unexpected end of input. Missing schema.");
- }
-
- if (result.getMessage().headerType() != MessageHeader.Schema) {
- throw new IOException(
- "Expected schema but header was " + result.getMessage().headerType());
- }
-
- schema = MessageSerializer.deserializeSchema(result.getMessage());
-
- }
-
- result = messageReader.readNext();
- if (result.getMessage().headerType() == MessageHeader.Schema) {
- result = messageReader.readNext();
- }
-
- if (result.getMessage().headerType() != MessageHeader.RecordBatch) {
- throw new IOException(
- "Expected recordbatch but header was " + result.getMessage().headerType());
- }
- var bodyBuffer = result.getBodyBuffer();
-
- // For zero-length batches, need an empty buffer to deserialize the batch
- if (bodyBuffer == null) {
- bodyBuffer = allocator.getEmpty();
- }
-
- val batch = MessageSerializer.deserializeRecordBatch(result.getMessage(), bodyBuffer);
- val vectors = fromArrowRecordBatch(schema, batch, allocator)
- val length = batch.getLength
- batch.close
- if (columnIndices == null) {
- new ColumnarBatch(vectors.map(_.asInstanceOf[ColumnVector]), length)
- } else {
- new ColumnarBatch(
- columnIndices.map(i => vectors(i).asInstanceOf[ColumnVector]),
- length)
- }
-
- } catch {
- case e: Throwable =>
- messageReader.close
- throw e
- }
- }
- }
- }
-
- def fromArrowRecordBatch(
- recordBatchSchema: Schema,
- recordBatch: ArrowRecordBatch,
- allocator: BufferAllocator = null): Array[ArrowWritableColumnVector] = {
- val numRows = recordBatch.getLength()
- ArrowWritableColumnVector.loadColumns(numRows, recordBatchSchema, recordBatch, allocator)
- }
-
- def releaseArrowRecordBatch(recordBatch: ArrowRecordBatch): Unit = {
- if (recordBatch != null) {
- recordBatch.close()
- }
- }
-
- def releaseArrowRecordBatchList(recordBatchList: Array[ArrowRecordBatch]): Unit = {
- recordBatchList.foreach({ recordBatch =>
- if (recordBatch != null)
- releaseArrowRecordBatch(recordBatch)
- })
- }
def getAttrFromExpr(fieldExpr: Expression, skipAlias: Boolean = false): AttributeReference = {
fieldExpr match {
@@ -340,6 +69,15 @@ object ConverterUtils extends Logging {
}
}
+ def getShortAttributeName(attr: Attribute): String = {
+ val subIndex = attr.name.indexOf("(")
+ if (subIndex != -1) {
+ attr.name.substring(0, subIndex)
+ } else {
+ attr.name
+ }
+ }
+
def getResultAttrFromExpr(
fieldExpr: Expression,
name: String = "None",
@@ -426,190 +164,10 @@ object ConverterUtils extends Logging {
true
}
- def combineArrowRecordBatch(rb1: ArrowRecordBatch, rb2: ArrowRecordBatch): ArrowRecordBatch = {
- val numRows = rb1.getLength()
- val rb1_nodes = rb1.getNodes()
- val rb2_nodes = rb2.getNodes()
- val rb1_bufferlist = rb1.getBuffers()
- val rb2_bufferlist = rb2.getBuffers()
-
- val combined_nodes = rb1_nodes.addAll(rb2_nodes)
- val combined_bufferlist = rb1_bufferlist.addAll(rb2_bufferlist)
- new ArrowRecordBatch(numRows, rb1_nodes, rb1_bufferlist)
- }
-
- def toArrowSchema(attributes: Seq[Attribute]): Schema = {
- val fields = attributes.map(attr => {
- Field
- .nullable(s"${attr.name}#${attr.exprId.id}", CodeGeneration.getResultType(attr.dataType))
- })
- new Schema(fields.toList.asJava)
- }
-
- def toArrowSchema(schema: StructType): Schema = {
- val fields = schema
- .map(field => {
- Field.nullable(field.name, CodeGeneration.getResultType(field.dataType))
- })
- new Schema(fields.toList.asJava)
- }
-
override def toString(): String = {
s"ConverterUtils"
}
- @throws[IOException]
- def getSchemaBytesBuf(schema: Schema): Array[Byte] = {
- val out: ByteArrayOutputStream = new ByteArrayOutputStream
- MessageSerializer.serialize(new WriteChannel(Channels.newChannel(out)), schema)
- out.toByteArray
- }
-
- @throws[IOException]
- def getSchemaFromBytesBuf(schema: Array[Byte]): Schema = {
- val in: ByteArrayInputStream = new ByteArrayInputStream(schema)
- MessageSerializer.deserializeSchema(new ReadChannel(Channels.newChannel(in)))
- }
-
- @throws[GandivaException]
- def getExprListBytesBuf(exprs: List[ExpressionTree]): Array[Byte] = {
- val builder: ExpressionList.Builder = GandivaTypes.ExpressionList.newBuilder
- exprs.foreach { expr => builder.addExprs(expr.toProtobuf) }
- builder.build.toByteArray
- }
-
- def checkIfTypeSupported(dt: DataType): Unit = dt match {
- case d: BooleanType =>
- case d: ByteType =>
- case d: ShortType =>
- case d: IntegerType =>
- case d: LongType =>
- case d: FloatType =>
- case d: DoubleType =>
- case d: StringType =>
- case d: DateType =>
- case d: DecimalType =>
- case d: TimestampType =>
- case _ =>
- throw new UnsupportedOperationException(s"Unsupported data type: $dt")
- }
-
- def createArrowField(name: String, dt: DataType): Field = dt match {
- case at: ArrayType =>
- new Field(
- name,
- FieldType.nullable(ArrowType.List.INSTANCE),
- Lists.newArrayList(createArrowField(s"${name}_${dt}", at.elementType)))
- case mt: MapType =>
- throw new UnsupportedOperationException(s"${dt} is not supported yet")
- case st: StructType =>
- throw new UnsupportedOperationException(s"${dt} is not supported yet")
- case _ =>
- Field.nullable(name, CodeGeneration.getResultType(dt))
- }
-
- def createArrowField(attr: Attribute): Field =
- createArrowField(s"${attr.name}#${attr.exprId.id}", attr.dataType)
-
- private def asTimestampType(inType: ArrowType): ArrowType.Timestamp = {
- if (inType.getTypeID != ArrowTypeID.Timestamp) {
- throw new IllegalArgumentException(s"Value type to convert must be timestamp")
- }
- inType.asInstanceOf[ArrowType.Timestamp]
- }
-
- def convertTimestampZone(inNode: TreeNode, inType: ArrowType,
- toZone: String): (TreeNode, ArrowType) = {
- throw new UnsupportedOperationException("not implemented") // fixme 20210602 hongze
- val inTimestamp = asTimestampType(inType)
- val fromZone = inTimestamp.getTimezone
-
- val (outNode0: TreeNode, outTimestamp0: ArrowType.Timestamp) =
- if (SparkSchemaUtils.timeZoneIDEquals(fromZone, toZone)) {
- val outType = new ArrowType.Timestamp(inTimestamp.getUnit, toZone)
- (inNode, outType)
- } else {
- // todo conversion
- }
- (outNode0, outTimestamp0)
- }
-
- def convertTimestampToMilli(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
- val inTimestamp = asTimestampType(inType)
- inTimestamp.getUnit match {
- case TimeUnit.MILLISECOND => (inNode, inType)
- case TimeUnit.MICROSECOND =>
- // truncate from micro to milli
- val outType = new ArrowType.Timestamp(TimeUnit.MILLISECOND,
- inTimestamp.getTimezone)
- (TreeBuilder.makeFunction(
- "convertTimestampUnit",
- Lists.newArrayList(inNode), outType), outType)
- }
- }
-
- def convertTimestampToMicro(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
- val inTimestamp = asTimestampType(inType)
- inTimestamp.getUnit match {
- case TimeUnit.MICROSECOND => (inNode, inType)
- case TimeUnit.MILLISECOND =>
- // truncate from micro to milli
- val outType = new ArrowType.Timestamp(TimeUnit.MICROSECOND,
- inTimestamp.getTimezone)
- (TreeBuilder.makeFunction(
- "convertTimestampUnit",
- Lists.newArrayList(inNode), outType), outType)
- }
- }
-
- def toInt32(inNode: TreeNode, inType: ArrowType): (TreeNode, ArrowType) = {
- val toType = ArrowUtils.toArrowType(IntegerType, null)
- val toNode = TreeBuilder.makeFunction("castINT", Lists.newArrayList(inNode),
- toType)
- (toNode, toType)
- }
-
- // use this carefully
- def toGandivaMicroUTCTimestamp(inNode: TreeNode, inType: ArrowType,
- timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
- val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get
- val utcTimestampNodeOriginal = inNode
- val inTimestampType = asTimestampType(inType)
- val inTimestampTypeUTC = new ArrowType.Timestamp(inTimestampType.getUnit,
- "UTC")
- ConverterUtils.convertTimestampToMicro(utcTimestampNodeOriginal,
- inTimestampTypeUTC)
- }
-
- // use this carefully
- def toGandivaTimestamp(inNode: TreeNode, inType: ArrowType,
- timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
- val zoneId = timeZoneId.orElse(Some(SparkSchemaUtils.getLocalTimezoneID())).get
-
- val utcTimestampNodeOriginal = inNode
- val utcTimestampNodeMilli = ConverterUtils.convertTimestampToMilli(utcTimestampNodeOriginal,
- inType)._1
- val utcTimestampNodeLong = TreeBuilder.makeFunction("castBIGINT",
- Lists.newArrayList(utcTimestampNodeMilli), new ArrowType.Int(64,
- true))
- val diff = SparkSchemaUtils.getTimeZoneIDOffset(zoneId) *
- DateTimeConstants.MILLIS_PER_SECOND
-
- val localizedTimestampNodeLong = TreeBuilder.makeFunction("add",
- Lists.newArrayList(utcTimestampNodeLong,
- TreeBuilder.makeLiteral(java.lang.Long.valueOf(diff))),
- new ArrowType.Int(64, true))
- val localized = new ArrowType.Timestamp(TimeUnit.MILLISECOND, null)
- val localizedTimestampNode = TreeBuilder.makeFunction("castTIMESTAMP",
- Lists.newArrayList(localizedTimestampNodeLong), localized)
- (localizedTimestampNode, localized)
- }
-
- def toSparkTimestamp(inNode: TreeNode, inType: ArrowType,
- timeZoneId: Option[String] = None): (TreeNode, ArrowType) = {
- throw new UnsupportedOperationException()
- }
-
def powerOfTen(pow: Int): (String, Int, Int) = {
val POWERS_OF_10: Array[(String, Int, Int)] = Array(
("1", 1, 0),
diff --git a/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala
index d06269721989..25e4be906844 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala
@@ -17,42 +17,11 @@
package io.glutenproject.expression
-import java.util.Collections
-
-import com.google.common.collect.Lists
-import io.glutenproject.expression.DateTimeExpressionsTransformer.castDateFromTimestamp
import io.glutenproject.substrait.expression.ExpressionNode
-import org.apache.arrow.gandiva.expression.TreeBuilder
-import org.apache.arrow.gandiva.expression.TreeNode
-import org.apache.arrow.vector.types.DateUnit
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.spark.sql.catalyst.expressions.CheckOverflow
-import org.apache.spark.sql.catalyst.expressions.CurrentDate
-import org.apache.spark.sql.catalyst.expressions.CurrentTimestamp
-import org.apache.spark.sql.catalyst.expressions.DateDiff
-import org.apache.spark.sql.catalyst.expressions.DayOfMonth
-import org.apache.spark.sql.catalyst.expressions.DayOfWeek
-import org.apache.spark.sql.catalyst.expressions.DayOfYear
-import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.expressions.Hour
-import org.apache.spark.sql.catalyst.expressions.MakeDate
-import org.apache.spark.sql.catalyst.expressions.MakeTimestamp
-import org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp
-import org.apache.spark.sql.catalyst.expressions.MillisToTimestamp
-import org.apache.spark.sql.catalyst.expressions.Minute
-import org.apache.spark.sql.catalyst.expressions.Month
-import org.apache.spark.sql.catalyst.expressions.Now
-import org.apache.spark.sql.catalyst.expressions.Second
-import org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp
-import org.apache.spark.sql.catalyst.expressions.UnixDate
-import org.apache.spark.sql.catalyst.expressions.UnixMicros
-import org.apache.spark.sql.catalyst.expressions.UnixMillis
-import org.apache.spark.sql.catalyst.expressions.UnixSeconds
-import org.apache.spark.sql.catalyst.expressions.UnixTimestamp
-import org.apache.spark.sql.catalyst.expressions.Year
+
+import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp, DateDiff, DayOfMonth, DayOfWeek, DayOfYear, Expression, Hour, MakeDate, MakeTimestamp, MicrosToTimestamp, MillisToTimestamp, Minute, Month, Now, Second, SecondsToTimestamp, UnixDate, UnixMicros, UnixMillis, UnixSeconds, UnixTimestamp, Year}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DateType, IntegerType, LongType, StringType, TimestampType}
-import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.sql.types.{DateType, TimestampType}
object DateTimeExpressionsTransformer {
class CurrentTimestampTransformer() extends CurrentTimestamp with ExpressionTransformer {
diff --git a/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala
index 1ef520613b5a..6903ea0dae5c 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/IfOperatorTransformer.scala
@@ -17,12 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
class IfTransformer(predicate: Expression, trueValue: Expression,
falseValue: Expression, original: Expression)
diff --git a/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala
index b2dff7ce29a4..74378fa584c6 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/InOperatorTransformer.scala
@@ -17,14 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
class InTransformer(value: Expression, list: Seq[Expression], original: Expression)
extends In(value: Expression, list: Seq[Expression])
diff --git a/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala
index 5ca8a8702c5d..73459a120cba 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/InSetOperatorTransformer.scala
@@ -17,17 +17,9 @@
package io.glutenproject.expression
-import java.lang
-
-import com.google.common.collect.Lists
-import com.google.common.collect.Sets
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
-
-import scala.collection.mutable.ListBuffer
class InSetTransformer(value: Expression, hset: Set[Any], original: Expression)
extends InSet(value: Expression, hset: Set[Any])
diff --git a/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala
index b51ffd16377b..c2e2aff6a79e 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/RoundOperatorTransformer.scala
@@ -17,14 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.optimizer._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
class RoundExpression(child: Expression, scale: Expression, original: Expression)
extends Round(child: Expression, scale: Expression)
diff --git a/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala
index d277bc875676..091c80c662bd 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/ScalarSubqueryTransformer.scala
@@ -17,25 +17,13 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
import io.glutenproject.substrait.expression.ExpressionNode
-import org.apache.arrow.gandiva.evaluator._
-import org.apache.arrow.gandiva.exceptions.GandivaException
-import org.apache.arrow.gandiva.expression._
-import org.apache.arrow.vector.types.DateUnit
-import org.apache.arrow.vector.types.pojo.ArrowType
-import org.apache.arrow.vector.types.pojo.Field
-import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.{InternalRow, expressions}
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
-import org.apache.spark.sql.execution.BaseSubqueryExec
-import org.apache.spark.sql.execution.ExecSubqueryExpression
import org.apache.spark.sql.execution.ScalarSubquery
import org.apache.spark.sql.types._
-import scala.collection.mutable.ListBuffer
-
class ScalarSubqueryTransformer(
query: ScalarSubquery)
extends Expression with ExpressionTransformer {
diff --git a/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala b/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala
index 9278642da3b0..6bb019715e05 100644
--- a/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala
+++ b/jvm/src/main/scala/io/glutenproject/expression/TernaryOperatorTransformer.scala
@@ -17,13 +17,9 @@
package io.glutenproject.expression
-import com.google.common.collect.Lists
import io.glutenproject.substrait.expression.ExpressionNode
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types._
-
-import scala.collection.mutable.ListBuffer
class SubStringTransformer(str: Expression, pos: Expression, len: Expression, original: Expression)
extends Substring(str: Expression, pos: Expression, len: Expression)
diff --git a/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala b/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala
index 2727d352d0d9..f7968e946ab7 100644
--- a/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala
+++ b/jvm/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala
@@ -18,9 +18,11 @@
package io.glutenproject.extension
import io.glutenproject.{GlutenConfig, GlutenSparkExtensionsInjector}
+import io.glutenproject.backendsapi.BackendsApiManager
import io.glutenproject.execution._
import io.glutenproject.extension.columnar.{RowGuard, TransformGuardRule}
import org.apache.spark.SparkConf
+
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
import org.apache.spark.sql.catalyst.expressions._
@@ -32,7 +34,6 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, V2CommandExec}
import org.apache.spark.sql.execution.exchange._
import org.apache.spark.sql.execution.joins._
-import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, ArrowEvalPythonExecTransformer}
import org.apache.spark.sql.execution.window.WindowExec
import org.apache.spark.sql.internal.SQLConf
@@ -49,9 +50,9 @@ case class TransformPreOverrides() extends Rule[SparkPlan] {
val actualPlan = plan.child
logDebug(s"Columnar Processing for ${actualPlan.getClass} is under RowGuard.")
actualPlan.withNewChildren(actualPlan.children.map(replaceWithTransformerPlan))
- case plan: ArrowEvalPythonExec =>
+ /* case plan: ArrowEvalPythonExec =>
val columnarChild = replaceWithTransformerPlan(plan.child)
- ArrowEvalPythonExecTransformer(plan.udfs, plan.resultAttrs, columnarChild, plan.evalType)
+ ArrowEvalPythonExecTransformer(plan.udfs, plan.resultAttrs, columnarChild, plan.evalType) */
case plan: BatchScanExec =>
logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.")
new BatchScanExecTransformer(plan.output, plan.scan)
@@ -205,7 +206,7 @@ case class TransformPostOverrides() extends Rule[SparkPlan] {
case plan: RowToColumnarExec =>
val child = replaceWithTransformerPlan(plan.child)
logDebug(s"ColumnarPostOverrides RowToArrowColumnarExec(${child.getClass})")
- RowToArrowColumnarExec(child)
+ BackendsApiManager.getSparkPlanExecApiInstance.genRowToArrowColumnarExec(child)
case ColumnarToRowExec(child: ColumnarShuffleExchangeAdaptor) =>
replaceWithTransformerPlan(child)
case ColumnarToRowExec(child: CoalesceBatchesExec) =>
@@ -214,8 +215,9 @@ case class TransformPostOverrides() extends Rule[SparkPlan] {
if (columnarConf.enableNativeColumnarToRow) {
val child = replaceWithTransformerPlan(plan.child)
logDebug(s"ColumnarPostOverrides NativeColumnarToRowExec(${child.getClass})")
- val nativeConversion = new NativeColumnarToRowExec(child)
- if (nativeConversion.doValidate()) {
+ val nativeConversion =
+ BackendsApiManager.getSparkPlanExecApiInstance.genNativeColumnarToRowExec(child)
+ if (nativeConversion.doValidate()) {
nativeConversion
} else {
logInfo("NativeColumnarToRow : Falling back to ColumnarToRow...")
@@ -234,7 +236,8 @@ case class TransformPostOverrides() extends Rule[SparkPlan] {
case c: ColumnarToRowExec =>
if (columnarConf.enableNativeColumnarToRow) {
val child = replaceWithTransformerPlan(c.child)
- val nativeConversion = new NativeColumnarToRowExec(child)
+ val nativeConversion =
+ BackendsApiManager.getSparkPlanExecApiInstance.genNativeColumnarToRowExec(child)
if (nativeConversion.doValidate()) {
nativeConversion
} else {
diff --git a/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala b/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala
index 178f7be60e7c..f75a0bda9f0a 100644
--- a/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala
+++ b/jvm/src/main/scala/io/glutenproject/extension/columnar/ColumnarGuardRule.scala
@@ -32,8 +32,6 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.exchange._
import org.apache.spark.sql.execution.joins._
-import org.apache.spark.sql.execution.python.ArrowEvalPythonExec
-import org.apache.spark.sql.execution.python.ArrowEvalPythonExecTransformer
import org.apache.spark.sql.execution.window.WindowExec
// A guard to prevent a plan being converted into the plan transformer.
@@ -75,11 +73,12 @@ case class TransformGuardRule() extends Rule[SparkPlan] {
private def tryConvertToTransformer(plan: SparkPlan): Boolean = {
try {
plan match {
- case plan: ArrowEvalPythonExec =>
+ /* case plan: ArrowEvalPythonExec =>
if (!enableColumnarArrowUDF) return false
val transformer = ArrowEvalPythonExecTransformer(
plan.udfs, plan.resultAttrs, plan.child, plan.evalType)
- transformer.doValidate()
+ if (!transformer.doValidate()) return false
+ transformer */
case plan: BatchScanExec =>
if (!enableColumnarBatchScan) return false
val transformer = new BatchScanExecTransformer(plan.output, plan.scan)
diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala b/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala
new file mode 100644
index 000000000000..afadee0aa502
--- /dev/null
+++ b/jvm/src/main/scala/org/apache/spark/shuffle/GlutenShuffleWriterWrapper.scala
@@ -0,0 +1,38 @@
+/*
+ * Copyright (2021) The Delta Lake Project Authors.
+ *
+ * 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.
+ */
+
+package org.apache.spark.shuffle
+
+import io.glutenproject.backendsapi.BackendsApiManager
+
+import org.apache.spark.shuffle.sort.ColumnarShuffleHandle
+
+case class GlutenShuffleWriterWrapper[K, V](shuffleWriter: ShuffleWriter[K, V])
+case class GenShuffleWriterParameters[K, V](shuffleBlockResolver: IndexShuffleBlockResolver,
+ columnarShuffleHandle: ColumnarShuffleHandle[K, V],
+ mapId: Long, metrics: ShuffleWriteMetricsReporter)
+
+object GlutenShuffleWriterWrapper {
+
+ def genColumnarShuffleWriter[K, V](shuffleBlockResolver: IndexShuffleBlockResolver,
+ columnarShuffleHandle: ColumnarShuffleHandle[K, V],
+ mapId: Long, metrics: ShuffleWriteMetricsReporter
+ ): ShuffleWriter[K, V] =
+ BackendsApiManager.getSparkPlanExecApiInstance
+ .genColumnarShuffleWriter(GenShuffleWriterParameters(shuffleBlockResolver,
+ columnarShuffleHandle, mapId, metrics))
+ .shuffleWriter
+}
\ No newline at end of file
diff --git a/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala b/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala
index fba719f96d13..788ab6d14a07 100644
--- a/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala
+++ b/jvm/src/main/scala/org/apache/spark/shuffle/sort/ColumnarShuffleManager.scala
@@ -20,7 +20,10 @@ package org.apache.spark.shuffle.sort
import java.io.InputStream
import java.util.concurrent.ConcurrentHashMap
-import org.apache.spark._
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv, TaskContext}
+
import org.apache.spark.internal.Logging
import org.apache.spark.serializer.SerializerManager
import org.apache.spark.shuffle._
@@ -29,8 +32,6 @@ import org.apache.spark.shuffle.sort.SortShuffleManager.canUseBatchFetch
import org.apache.spark.storage.BlockId
import org.apache.spark.util.collection.OpenHashSet
-import scala.collection.JavaConverters._
-
class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Logging {
import ColumnarShuffleManager._
@@ -48,8 +49,8 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin
* Obtains a [[ShuffleHandle]] to pass to tasks.
*/
override def registerShuffle[K, V, C](
- shuffleId: Int,
- dependency: ShuffleDependency[K, V, C]): ShuffleHandle = {
+ shuffleId: Int,
+ dependency: ShuffleDependency[K, V, C]): ShuffleHandle = {
if (dependency.isInstanceOf[ColumnarShuffleDependency[_, _, _]]) {
logInfo(s"Registering ColumnarShuffle shuffleId: ${shuffleId}")
new ColumnarShuffleHandle[K, V](
@@ -77,17 +78,18 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin
/** Get a writer for a given partition. Called on executors by map tasks. */
override def getWriter[K, V](
- handle: ShuffleHandle,
- mapId: Long,
- context: TaskContext,
- metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = {
+ handle: ShuffleHandle,
+ mapId: Long,
+ context: TaskContext,
+ metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = {
val mapTaskIds =
taskIdMapsForShuffle.computeIfAbsent(handle.shuffleId, _ => new OpenHashSet[Long](16))
mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) }
val env = SparkEnv.get
handle match {
case columnarShuffleHandle: ColumnarShuffleHandle[K @unchecked, V @unchecked] =>
- new ColumnarShuffleWriter(shuffleBlockResolver, columnarShuffleHandle, mapId, metrics)
+ GlutenShuffleWriterWrapper.genColumnarShuffleWriter(
+ shuffleBlockResolver, columnarShuffleHandle, mapId, metrics)
case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] =>
new UnsafeShuffleWriter(
env.blockManager,
@@ -121,13 +123,13 @@ class ColumnarShuffleManager(conf: SparkConf) extends ShuffleManager with Loggin
* Called on executors by reduce tasks.
*/
override def getReader[K, C](
- handle: ShuffleHandle,
- startMapIndex: Int,
- endMapIndex: Int,
- startPartition: Int,
- endPartition: Int,
- context: TaskContext,
- metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = {
+ handle: ShuffleHandle,
+ startMapIndex: Int,
+ endMapIndex: Int,
+ startPartition: Int,
+ endPartition: Int,
+ context: TaskContext,
+ metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = {
val blocksByAddress = SparkEnv.get.mapOutputTracker
.getMapSizesByExecutorId(handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition)
if (handle.isInstanceOf[ColumnarShuffleHandle[K, _]]) {
@@ -189,6 +191,6 @@ object ColumnarShuffleManager extends Logging {
}
private[spark] class ColumnarShuffleHandle[K, V](
- shuffleId: Int,
- dependency: ShuffleDependency[K, V, V])
- extends BaseShuffleHandle(shuffleId, dependency) {}
+ shuffleId: Int,
+ dependency: ShuffleDependency[K, V, V])
+ extends BaseShuffleHandle(shuffleId, dependency) {}
diff --git a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala
index c9dd2c374067..a3e5bd1d2076 100644
--- a/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala
+++ b/jvm/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala
@@ -20,33 +20,20 @@ package org.apache.spark.sql.execution
import scala.collection.JavaConverters._
import scala.concurrent.Future
-import io.glutenproject.GlutenConfig
-import io.glutenproject.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer}
-import io.glutenproject.substrait.expression.ExpressionNode
-import io.glutenproject.substrait.rel.RelBuilder
-import io.glutenproject.vectorized.{ArrowWritableColumnVector, ColumnarFactory, NativePartitioning}
-import java.util
-import org.apache.arrow.gandiva.expression.TreeNode
-import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+import io.glutenproject.backendsapi.BackendsApiManager
import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.Serializer
-import org.apache.spark.shuffle.{ColumnarShuffleDependency, ShuffleHandle}
+import org.apache.spark.shuffle.ShuffleHandle
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection}
-import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
+import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.CoalesceExec.EmptyPartition
-import org.apache.spark.sql.execution.datasources.v2.arrow.SparkMemoryUtils
import org.apache.spark.sql.execution.exchange._
-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.createShuffleWriteProcessor
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter}
-import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
-import org.apache.spark.util.MutablePair
case class ColumnarShuffleExchangeExec(override val outputPartitioning: Partitioning,
child: SparkPlan,
@@ -81,33 +68,13 @@ case class ColumnarShuffleExchangeExec(override val outputPartitioning: Partitio
// super.stringArgs ++ Iterator(output.map(o => s"${o}#${o.dataType.simpleString}"))
def doValidate(): Boolean = {
- if (GlutenConfig.getConf.isClickHouseBackend) return true
- // check input datatype
- for (attr <- child.output) {
- try {
- ConverterUtils.createArrowField(attr)
- } catch {
- case e: UnsupportedOperationException =>
- logInfo(s"${attr.dataType} is not supported in ColumnarShuffledExchangeExec.")
- return false
- }
- }
- outputPartitioning match {
- case HashPartitioning(exprs, n) =>
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- exprs.foreach(expr => {
- if (!expr.isInstanceOf[Attribute]) {
- logInfo("Expressions are not supported in HashPartitioning.")
- return false
- }})
- }
- case _ =>
- }
- true
+ BackendsApiManager.getTransformerApiInstance.validateColumnarShuffleExchangeExec(
+ outputPartitioning, child.output)
}
- val serializer: Serializer = ColumnarFactory.createColumnarBatchSerializer(
- schema,
+ val serializer: Serializer =
+ BackendsApiManager.getSparkPlanExecApiInstance
+ .createColumnarBatchSerializer(schema,
longMetric("avgReadBatchNumRows"),
longMetric("numOutputRows"))
@@ -215,8 +182,8 @@ class ColumnarShuffleExchangeAdaptor(override val outputPartitioning: Partitioni
super.stringArgs ++ Iterator(s"[id=#$id]")
//super.stringArgs ++ Iterator(output.map(o => s"${o}#${o.dataType.simpleString}"))
- val serializer: Serializer = ColumnarFactory.createColumnarBatchSerializer(
- schema,
+ val serializer: Serializer = BackendsApiManager.getSparkPlanExecApiInstance
+ .createColumnarBatchSerializer(schema,
longMetric("avgReadBatchNumRows"),
longMetric("numOutputRows"))
@@ -328,196 +295,18 @@ object ColumnarShuffleExchangeExec extends Logging {
spillTime: SQLMetric,
compressTime: SQLMetric,
prepareTime: SQLMetric): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = {
- val arrowFields = outputAttributes.map(attr => ConverterUtils.createArrowField(attr))
- def serializeSchema(fields: Seq[Field]): Array[Byte] = {
- val schema = new Schema(fields.asJava)
- ConverterUtils.getSchemaBytesBuf(schema)
- }
-
- // only used for fallback range partitioning
- val rangePartitioner: Option[Partitioner] = newPartitioning match {
- case RangePartitioning(sortingExpressions, numPartitions) =>
- // Extract only fields used for sorting to avoid collecting large fields that does not
- // affect sorting result when deciding partition bounds in RangePartitioner
- val rddForSampling = rdd.mapPartitionsInternal { iter =>
- // Internally, RangePartitioner runs a job on the RDD that samples keys to compute
- // partition bounds. To get accurate samples, we need to copy the mutable keys.
- iter.flatMap(batch => {
- val rows = batch.rowIterator.asScala
- val projection =
- UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes)
- val mutablePair = new MutablePair[InternalRow, Null]()
- rows.map(row => mutablePair.update(projection(row).copy(), null))
- })
- }
- // Construct ordering on extracted sort key.
- val orderingAttributes = sortingExpressions.zipWithIndex.map {
- case (ord, i) =>
- ord.copy(child = BoundReference(i, ord.dataType, ord.nullable))
- }
- implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes)
- val part = new RangePartitioner(
- numPartitions,
- rddForSampling,
- ascending = true,
- samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
- Some(part)
- case _ => None
- }
-
- // only used for fallback range partitioning
- def computeAndAddPartitionId(
- cbIter: Iterator[ColumnarBatch],
- partitionKeyExtractor: InternalRow => Any): CloseablePairedColumnarBatchIterator = {
- CloseablePairedColumnarBatchIterator {
- cbIter
- .filter(cb => cb.numRows != 0 && cb.numCols != 0)
- .map { cb =>
- val startTime = System.nanoTime()
- val pidVec = ArrowWritableColumnVector
- .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType))
- .head
- (0 until cb.numRows).foreach { i =>
- val row = cb.getRow(i)
- val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row))
- pidVec.putInt(i, pid)
- }
-
- val newColumns = (pidVec +: (0 until cb.numCols).map(cb.column)).toArray
- newColumns.foreach(
- _.asInstanceOf[ArrowWritableColumnVector].getValueVector.setValueCount(cb.numRows))
- computePidTime.add(System.nanoTime() - startTime)
- (0, new ColumnarBatch(newColumns, cb.numRows))
- }
- }
- }
-
- val nativePartitioning: NativePartitioning = newPartitioning match {
- case SinglePartition => new NativePartitioning("single", 1, serializeSchema(arrowFields))
- case RoundRobinPartitioning(n) =>
- new NativePartitioning("rr", n, serializeSchema(arrowFields))
- case HashPartitioning(exprs, n) =>
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- // Function map is not expected to be used.
- val functionMap = new java.util.HashMap[String, java.lang.Long]()
- val exprNodeList = new util.ArrayList[ExpressionNode]()
- exprs.foreach(expr => {
- exprNodeList.add(ExpressionConverter
- .replaceWithExpressionTransformer(expr, outputAttributes)
- .asInstanceOf[ExpressionTransformer]
- .doTransform(functionMap))
- })
- val projectRel = RelBuilder.makeProjectRel(null, exprNodeList)
- new NativePartitioning(
- "hash",
- n,
- serializeSchema(arrowFields),
- projectRel.toProtobuf().toByteArray)
- } else {
- val fields = exprs.zipWithIndex.map {
- case (expr, i) =>
- val treeNode: TreeNode = null
- val attr = ConverterUtils.getAttrFromExpr(expr)
- attr.name
- }
- new NativePartitioning(
- "hash",
- n,
- null,
- fields.mkString(",").getBytes("UTF-8"))
- }
- // range partitioning fall back to row-based partition id computation
- case RangePartitioning(orders, n) =>
- val pidField = Field.nullable("pid", new ArrowType.Int(32, true))
- new NativePartitioning("range", n, serializeSchema(pidField +: arrowFields))
- }
-
- val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] &&
- newPartitioning.numPartitions > 1
-
- // RDD passed to ShuffleDependency should be the form of key-value pairs.
- // ColumnarShuffleWriter will compute ids from ColumnarBatch on native side other than read the "key" part.
- // Thus in Columnar Shuffle we never use the "key" part.
- val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition
-
- val rddWithDummyKey: RDD[Product2[Int, ColumnarBatch]] = newPartitioning match {
- case RangePartitioning(sortingExpressions, _) =>
- rdd.mapPartitionsWithIndexInternal((_, cbIter) => {
- val partitionKeyExtractor: InternalRow => Any = {
- val projection =
- UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes)
- row => projection(row)
- }
- val newIter = computeAndAddPartitionId(cbIter, partitionKeyExtractor)
-
- SparkMemoryUtils.addLeakSafeTaskCompletionListener[Unit] { _ =>
- newIter.closeAppendedVector()
- }
-
- newIter
- }, isOrderSensitive = isOrderSensitive)
- case _ =>
- rdd.mapPartitionsWithIndexInternal(
- (_, cbIter) =>
- cbIter.map { cb =>
- if (!GlutenConfig.getConf.isClickHouseBackend) {
- (0 until cb.numCols).foreach(
- cb.column(_)
- .asInstanceOf[ArrowWritableColumnVector]
- .getValueVector
- .setValueCount(cb.numRows))
- }
- (0, cb)
- },
- isOrderSensitive = isOrderSensitive)
- }
-
- val dependency =
- new ColumnarShuffleDependency[Int, ColumnarBatch, ColumnarBatch](
- rddWithDummyKey,
- new PartitionIdPassthrough(newPartitioning.numPartitions),
- serializer,
- shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics),
- nativePartitioning = nativePartitioning,
- dataSize = dataSize,
- bytesSpilled = bytesSpilled,
- numInputRows = numInputRows,
- computePidTime = computePidTime,
- splitTime = splitTime,
- spillTime = spillTime,
- compressTime = compressTime,
- prepareTime = prepareTime)
-
- dependency
- }
-}
-
-case class CloseablePairedColumnarBatchIterator(iter: Iterator[(Int, ColumnarBatch)])
- extends Iterator[(Int, ColumnarBatch)]
- with Logging {
-
- private var cur: (Int, ColumnarBatch) = _
-
- def closeAppendedVector(): Unit = {
- if (cur != null) {
- logDebug("Close appended partition id vector")
- cur match {
- case (_, cb: ColumnarBatch) =>
- cb.column(0).asInstanceOf[ArrowWritableColumnVector].close()
- }
- cur = null
- }
- }
-
- override def hasNext: Boolean = {
- iter.hasNext
- }
-
- override def next(): (Int, ColumnarBatch) = {
- closeAppendedVector()
- if (iter.hasNext) {
- cur = iter.next()
- cur
- } else Iterator.empty.next()
+ BackendsApiManager.getSparkPlanExecApiInstance.genShuffleDependency(rdd,
+ outputAttributes,
+ newPartitioning: Partitioning,
+ serializer: Serializer,
+ writeMetrics,
+ dataSize: SQLMetric,
+ bytesSpilled: SQLMetric,
+ numInputRows: SQLMetric,
+ computePidTime: SQLMetric,
+ splitTime: SQLMetric,
+ spillTime: SQLMetric,
+ compressTime: SQLMetric,
+ prepareTime: SQLMetric)
}
}
diff --git a/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java b/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java
new file mode 100644
index 000000000000..0ff7ec03d49e
--- /dev/null
+++ b/jvm/src/test/java/io/glutenproject/backendsapi/TestBackendsApisSuite.java
@@ -0,0 +1,37 @@
+/*
+ * 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 io.glutenproject.backendsapi;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestBackendsApisSuite {
+
+ @Test
+ public void testInitializeBackendsApi() {
+ BackendsApiManager.initialize("default");
+ Assert.assertNotNull(BackendsApiManager.getIteratorApiInstance());
+ Assert.assertNotNull(BackendsApiManager.getTransformerApiInstance());
+ Assert.assertNotNull(BackendsApiManager.getSparkPlanExecApiInstance());
+ }
+
+ @Test(expected = AssertionError.class)
+ public void testNullBackendsApi() {
+ BackendsApiManager.initialize("none");
+ }
+}
diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi
new file mode 100644
index 000000000000..5de95b463c31
--- /dev/null
+++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.IIteratorApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.IteratorApiImplSuite
\ No newline at end of file
diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi
new file mode 100644
index 000000000000..67062568046c
--- /dev/null
+++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ISparkPlanExecApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.SparkPlanExecApiImplSuite
\ No newline at end of file
diff --git a/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi
new file mode 100644
index 000000000000..2eef799617c7
--- /dev/null
+++ b/jvm/src/test/resources/META-INF/services/io.glutenproject.backendsapi.ITransformerApi
@@ -0,0 +1 @@
+io.glutenproject.backendsapi.TransformerApiImplSuite
\ No newline at end of file
diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala
new file mode 100644
index 000000000000..10e5ef110671
--- /dev/null
+++ b/jvm/src/test/scala/io/glutenproject/backendsapi/IteratorApiImplSuite.scala
@@ -0,0 +1,114 @@
+/*
+ * 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 io.glutenproject.backendsapi
+
+import io.glutenproject.GlutenNumaBindingInfo
+import io.glutenproject.execution.AbstractColumnarNativeIterator
+import io.glutenproject.execution.BaseNativeFilePartition
+import io.glutenproject.execution.WholestageTransformContext
+import io.glutenproject.substrait.plan.PlanNode
+import io.glutenproject.vectorized.AbstractBatchIterator
+import io.glutenproject.vectorized.ExpressionEvaluator
+import io.glutenproject.vectorized.ExpressionEvaluatorJniWrapper
+import org.apache.spark.SparkConf
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.connector.read.InputPartition
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+class IteratorApiImplSuite extends IIteratorApi {
+ /**
+ * Generate native row partition.
+ *
+ * @return
+ */
+ override def genNativeFilePartition(p: InputPartition,
+ wsCxt: WholestageTransformContext
+ ): BaseNativeFilePartition = null
+
+ /**
+ * Generate Iterator[ColumnarBatch] for CoalesceBatchesExec.
+ *
+ * @return
+ */
+ override def genCoalesceIterator(iter: Iterator[ColumnarBatch], recordsPerBatch: Int,
+ numOutputRows: SQLMetric, numInputBatches: SQLMetric,
+ numOutputBatches: SQLMetric, collectTime: SQLMetric,
+ concatTime: SQLMetric, avgCoalescedNumRows: SQLMetric
+ ): Iterator[ColumnarBatch] = null
+
+ /**
+ * Generate closeable ColumnBatch iterator.
+ *
+ * @return
+ */
+ override def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]
+ ): Iterator[ColumnarBatch] = null
+
+ /**
+ * Generate Iterator[ColumnarBatch] for first stage.
+ *
+ * @return
+ */
+ override def genFirstStageIterator(inputPartition: BaseNativeFilePartition,
+ loadNative: Boolean, outputAttributes: Seq[Attribute],
+ context: TaskContext, jarList: Seq[String]
+ ): Iterator[ColumnarBatch] = null
+
+ /**
+ * Generate Iterator[ColumnarBatch] for final stage.
+ *
+ * @return
+ */
+ override def genFinalStageIterator(iter: Iterator[ColumnarBatch],
+ numaBindingInfo: GlutenNumaBindingInfo,
+ listJars: Seq[String], signature: String,
+ sparkConf: SparkConf, outputAttributes: Seq[Attribute],
+ rootNode: PlanNode, streamedSortPlan: SparkPlan,
+ pipelineTime: SQLMetric, buildRelationBatchHolder: Seq[ColumnarBatch],
+ dependentKernels: Seq[ExpressionEvaluator],
+ dependentKernelIterators: Seq[AbstractBatchIterator]
+ ): Iterator[ColumnarBatch] = null
+
+ /**
+ * Generate columnar native iterator.
+ *
+ * @return
+ */
+ override def genColumnarNativeIterator(delegated: Iterator[ColumnarBatch]
+ ): AbstractColumnarNativeIterator = null
+
+ /**
+ * Generate BatchIterator for ExpressionEvaluator.
+ *
+ * @return
+ */
+ override def genBatchIterator(wsPlan: Array[Byte],
+ iterList: Seq[AbstractColumnarNativeIterator],
+ jniWrapper: ExpressionEvaluatorJniWrapper
+ ): AbstractBatchIterator = null
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = "default"
+}
diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala
new file mode 100644
index 000000000000..ee262c128e01
--- /dev/null
+++ b/jvm/src/test/scala/io/glutenproject/backendsapi/SparkPlanExecApiImplSuite.scala
@@ -0,0 +1,87 @@
+/*
+ * 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 io.glutenproject.backendsapi
+
+import io.glutenproject.execution.{NativeColumnarToRowExec, RowToArrowColumnarExec}
+import org.apache.spark.ShuffleDependency
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.shuffle.{GenShuffleWriterParameters, GlutenShuffleWriterWrapper}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+class SparkPlanExecApiImplSuite extends ISparkPlanExecApi {
+ /**
+ * Generate NativeColumnarToRowExec.
+ *
+ * @param child
+ * @return
+ */
+ override def genNativeColumnarToRowExec(child: SparkPlan): NativeColumnarToRowExec = null
+
+ /**
+ * Generate RowToArrowColumnarExec.
+ *
+ * @param child
+ * @return
+ */
+ override def genRowToArrowColumnarExec(child: SparkPlan): RowToArrowColumnarExec = null
+
+ /**
+ * Generate ShuffleDependency for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def genShuffleDependency(rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute],
+ newPartitioning: Partitioning, serializer: Serializer,
+ writeMetrics: Map[String, SQLMetric], dataSize: SQLMetric,
+ bytesSpilled: SQLMetric, numInputRows: SQLMetric,
+ computePidTime: SQLMetric, splitTime: SQLMetric,
+ spillTime: SQLMetric, compressTime: SQLMetric,
+ prepareTime: SQLMetric
+ ): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = null
+
+ /**
+ * Generate ColumnarShuffleWriter for ColumnarShuffleManager.
+ *
+ * @return
+ */
+ override def genColumnarShuffleWriter[K, V](parameters: GenShuffleWriterParameters[K, V]
+ ): GlutenShuffleWriterWrapper[K, V] = null
+
+ /**
+ * Generate ColumnarBatchSerializer for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def createColumnarBatchSerializer(schema: StructType,
+ readBatchNumRows: SQLMetric,
+ numOutputRows: SQLMetric): Serializer = null
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = "default"
+}
diff --git a/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala b/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala
new file mode 100644
index 000000000000..ddf01467aae1
--- /dev/null
+++ b/jvm/src/test/scala/io/glutenproject/backendsapi/TransformerApiImplSuite.scala
@@ -0,0 +1,40 @@
+/*
+ * 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 io.glutenproject.backendsapi
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+
+class TransformerApiImplSuite extends ITransformerApi {
+
+ /**
+ * Do validate for ColumnarShuffleExchangeExec.
+ *
+ * @return
+ */
+ override def validateColumnarShuffleExchangeExec(outputPartitioning: Partitioning,
+ outputAttributes: Seq[Attribute]
+ ): Boolean = false
+
+ /**
+ * Get the backend api name.
+ *
+ * @return
+ */
+ override def getBackendName: String = "default"
+}
diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala b/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala
index 2dc971feed52..1cb6feac6044 100644
--- a/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala
+++ b/jvm/src/test/scala/io/glutenproject/benchmarks/BenchmarkTest.scala
@@ -72,7 +72,7 @@ object BenchmarkTest {
.config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true")
.config(GlutenConfig.GLUTEN_LOAD_ARROW, "false")
.config(GlutenConfig.GLUTEN_LIB_PATH,
- "/home/myubuntu/Works/c_cpp_projects/Kyligence-ClickHouse-MergeTree/cmake-build-release/utils/local-engine/liblocal_engine_jni.so")
+ "/usr/local/clickhouse/lib/libch.so")
.config("spark.gluten.sql.columnar.iterator", "false")
.config("spark.gluten.sql.columnar.ch.mergetree.enabled", "true")
.config("spark.gluten.sql.columnar.ch.mergetree.table.path",
diff --git a/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala b/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala
index 9fe4fa487d9a..4aa594604b89 100644
--- a/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala
+++ b/jvm/src/test/scala/io/glutenproject/benchmarks/TableBenchmarkTest.scala
@@ -93,7 +93,7 @@ object TableBenchmarkTest {
.config(GlutenConfig.GLUTEN_LOAD_NATIVE, "true")
.config(GlutenConfig.GLUTEN_LOAD_ARROW, "false")
.config(GlutenConfig.GLUTEN_LIB_PATH,
- "/home/myubuntu/Works/c_cpp_projects/Kyligence-ClickHouse-MergeTree/cmake-build-release/utils/local-engine/liblocal_engine_jni.so")
+ "/usr/local/clickhouse/lib/libch.so")
.config("spark.gluten.sql.columnar.iterator", "false")
.config("spark.gluten.sql.columnar.ch.mergetree.enabled", "true")
.config("spark.gluten.sql.columnar.ch.mergetree.table.path",
diff --git a/pom.xml b/pom.xml
index 7b234d51e891..e489020a04d7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -34,6 +34,34 @@
shims
+
+ 2.12.10
+ 2.12
+ 3.1.1
+ 8.0.0-gluten-SNAPSHOT
+ arrow-memory-unsafe
+ ${hadoop.version}
+ UTF-8
+ UTF-8
+ ${project.basedir}/../tools
+ ${project.basedir}/../tools
+ OFF
+ OFF
+ OFF
+ OFF
+ /usr/local
+ ON
+ ON
+ OFF
+ OFF
+ OFF
+ ${project.basedir}/../tools/build_velox/velox_ep
+ spark-sql-columnar
+ Gluten
+ gluten
+ 3.1.1
+
+
spark
@@ -107,34 +135,35 @@
arrow-memory-netty
-
-
- 2.12.10
- 2.12
- 3.1.1
- 8.0.0-gluten-SNAPSHOT
- arrow-memory-unsafe
- ${hadoop.version}
- UTF-8
- UTF-8
- ${project.basedir}/../tools
- ${project.basedir}/../tools
- OFF
- OFF
- OFF
- OFF
- /usr/local
- ON
- ON
- OFF
- OFF
- OFF
- ${project.basedir}/../tools/build_velox/velox_ep
- spark-sql-columnar
- Gluten
- 3.1.1
-
+
+ backends-clickhouse
+
+ false
+
+
+ backends-clickhouse
+
+
+
+ backends-velox
+
+ false
+
+
+ backends-velox
+
+
+
+ backends-gazelle
+
+ false
+
+
+ backends-velox
+
+
+