Skip to content

Commit 1fbdbc4

Browse files
authored
[GLUTEN-6067][CH] [Part 2] Support CH backend with Spark3.5 - Prepare for supporting sink transform (#6197)
[CH] [Part 2] Support CH backend with Spark3.5 - Prepare for supporting sink transform * [Refactor] remove duplicate codes * Add NativeWriteChecker * [Prepare to commit] getExtendedColumnarPostRules from Spark shim
1 parent f07e348 commit 1fbdbc4

22 files changed

+1379
-1055
lines changed

backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala

+70-73
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
*/
1717
package org.apache.gluten.backendsapi.clickhouse
1818

19-
import org.apache.gluten.{GlutenConfig, GlutenNumaBindingInfo}
19+
import org.apache.gluten.GlutenNumaBindingInfo
2020
import org.apache.gluten.backendsapi.IteratorApi
2121
import org.apache.gluten.execution._
2222
import org.apache.gluten.expression.ConverterUtils
@@ -61,6 +61,52 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
6161
StructType(dataSchema)
6262
}
6363

64+
private def createNativeIterator(
65+
splitInfoByteArray: Array[Array[Byte]],
66+
wsPlan: Array[Byte],
67+
materializeInput: Boolean,
68+
inputIterators: Seq[Iterator[ColumnarBatch]]): BatchIterator = {
69+
70+
/** Generate closeable ColumnBatch iterator. */
71+
val listIterator =
72+
inputIterators
73+
.map {
74+
case i: CloseableCHColumnBatchIterator => i
75+
case it => new CloseableCHColumnBatchIterator(it)
76+
}
77+
.map(it => new ColumnarNativeIterator(it.asJava).asInstanceOf[GeneralInIterator])
78+
.asJava
79+
new CHNativeExpressionEvaluator().createKernelWithBatchIterator(
80+
wsPlan,
81+
splitInfoByteArray,
82+
listIterator,
83+
materializeInput
84+
)
85+
}
86+
87+
private def createCloseIterator(
88+
context: TaskContext,
89+
pipelineTime: SQLMetric,
90+
updateNativeMetrics: IMetrics => Unit,
91+
updateInputMetrics: Option[InputMetricsWrapper => Unit] = None,
92+
nativeIter: BatchIterator): CloseableCHColumnBatchIterator = {
93+
94+
val iter = new CollectMetricIterator(
95+
nativeIter,
96+
updateNativeMetrics,
97+
updateInputMetrics,
98+
updateInputMetrics.map(_ => context.taskMetrics().inputMetrics).orNull)
99+
100+
context.addTaskFailureListener(
101+
(ctx, _) => {
102+
if (ctx.isInterrupted()) {
103+
iter.cancel()
104+
}
105+
})
106+
context.addTaskCompletionListener[Unit](_ => iter.close())
107+
new CloseableCHColumnBatchIterator(iter, Some(pipelineTime))
108+
}
109+
64110
// only set file schema for text format table
65111
private def setFileSchemaForLocalFiles(
66112
localFilesNode: LocalFilesNode,
@@ -198,45 +244,24 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
198244
inputIterators: Seq[Iterator[ColumnarBatch]] = Seq()
199245
): Iterator[ColumnarBatch] = {
200246

201-
assert(
247+
require(
202248
inputPartition.isInstanceOf[GlutenPartition],
203249
"CH backend only accepts GlutenPartition in GlutenWholeStageColumnarRDD.")
204-
205-
val transKernel = new CHNativeExpressionEvaluator()
206-
val inBatchIters = new JArrayList[GeneralInIterator](inputIterators.map {
207-
iter => new ColumnarNativeIterator(CHIteratorApi.genCloseableColumnBatchIterator(iter).asJava)
208-
}.asJava)
209-
210250
val splitInfoByteArray = inputPartition
211251
.asInstanceOf[GlutenPartition]
212252
.splitInfosByteArray
213-
val nativeIter =
214-
transKernel.createKernelWithBatchIterator(
215-
inputPartition.plan,
216-
splitInfoByteArray,
217-
inBatchIters,
218-
false)
253+
val wsPlan = inputPartition.plan
254+
val materializeInput = false
219255

220-
val iter = new CollectMetricIterator(
221-
nativeIter,
222-
updateNativeMetrics,
223-
updateInputMetrics,
224-
context.taskMetrics().inputMetrics)
225-
226-
context.addTaskFailureListener(
227-
(ctx, _) => {
228-
if (ctx.isInterrupted()) {
229-
iter.cancel()
230-
}
231-
})
232-
context.addTaskCompletionListener[Unit](_ => iter.close())
233-
234-
// TODO: SPARK-25083 remove the type erasure hack in data source scan
235256
new InterruptibleIterator(
236257
context,
237-
new CloseableCHColumnBatchIterator(
238-
iter.asInstanceOf[Iterator[ColumnarBatch]],
239-
Some(pipelineTime)))
258+
createCloseIterator(
259+
context,
260+
pipelineTime,
261+
updateNativeMetrics,
262+
Some(updateInputMetrics),
263+
createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators))
264+
)
240265
}
241266

242267
// Generate Iterator[ColumnarBatch] for final stage.
@@ -252,52 +277,26 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
252277
partitionIndex: Int,
253278
materializeInput: Boolean): Iterator[ColumnarBatch] = {
254279
// scalastyle:on argcount
255-
GlutenConfig.getConf
256-
257-
val transKernel = new CHNativeExpressionEvaluator()
258-
val columnarNativeIterator =
259-
new JArrayList[GeneralInIterator](inputIterators.map {
260-
iter =>
261-
new ColumnarNativeIterator(CHIteratorApi.genCloseableColumnBatchIterator(iter).asJava)
262-
}.asJava)
263-
// we need to complete dependency RDD's firstly
264-
val nativeIterator = transKernel.createKernelWithBatchIterator(
265-
rootNode.toProtobuf.toByteArray,
266-
// Final iterator does not contain scan split, so pass empty split info to native here.
267-
new Array[Array[Byte]](0),
268-
columnarNativeIterator,
269-
materializeInput
270-
)
271-
272-
val iter = new CollectMetricIterator(nativeIterator, updateNativeMetrics, null, null)
273280

274-
context.addTaskFailureListener(
275-
(ctx, _) => {
276-
if (ctx.isInterrupted()) {
277-
iter.cancel()
278-
}
279-
})
280-
context.addTaskCompletionListener[Unit](_ => iter.close())
281-
new CloseableCHColumnBatchIterator(iter, Some(pipelineTime))
282-
}
283-
}
281+
// Final iterator does not contain scan split, so pass empty split info to native here.
282+
val splitInfoByteArray = new Array[Array[Byte]](0)
283+
val wsPlan = rootNode.toProtobuf.toByteArray
284284

285-
object CHIteratorApi {
286-
287-
/** Generate closeable ColumnBatch iterator. */
288-
def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = {
289-
iter match {
290-
case _: CloseableCHColumnBatchIterator => iter
291-
case _ => new CloseableCHColumnBatchIterator(iter)
292-
}
285+
// we need to complete dependency RDD's firstly
286+
createCloseIterator(
287+
context,
288+
pipelineTime,
289+
updateNativeMetrics,
290+
None,
291+
createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators))
293292
}
294293
}
295294

296295
class CollectMetricIterator(
297296
val nativeIterator: BatchIterator,
298297
val updateNativeMetrics: IMetrics => Unit,
299-
val updateInputMetrics: InputMetricsWrapper => Unit,
300-
val inputMetrics: InputMetrics
298+
val updateInputMetrics: Option[InputMetricsWrapper => Unit] = None,
299+
val inputMetrics: InputMetrics = null
301300
) extends Iterator[ColumnarBatch] {
302301
private var outputRowCount = 0L
303302
private var outputVectorCount = 0L
@@ -329,9 +328,7 @@ class CollectMetricIterator(
329328
val nativeMetrics = nativeIterator.getMetrics.asInstanceOf[NativeMetrics]
330329
nativeMetrics.setFinalOutputMetrics(outputRowCount, outputVectorCount)
331330
updateNativeMetrics(nativeMetrics)
332-
if (updateInputMetrics != null) {
333-
updateInputMetrics(inputMetrics)
334-
}
331+
updateInputMetrics.foreach(_(inputMetrics))
335332
metricsUpdated = true
336333
}
337334
}

backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala

-9
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,6 @@ import org.apache.spark.sql.delta.files.TahoeFileIndex
5050
import org.apache.spark.sql.execution._
5151
import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec
5252
import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation}
53-
import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.NativeWritePostRule
5453
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
5554
import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat
5655
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec}
@@ -583,14 +582,6 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
583582
override def genExtendedColumnarTransformRules(): List[SparkSession => Rule[SparkPlan]] =
584583
List()
585584

586-
/**
587-
* Generate extended columnar post-rules.
588-
*
589-
* @return
590-
*/
591-
override def genExtendedColumnarPostRules(): List[SparkSession => Rule[SparkPlan]] =
592-
List(spark => NativeWritePostRule(spark))
593-
594585
override def genInjectPostHocResolutionRules(): List[SparkSession => Rule[LogicalPlan]] = {
595586
List()
596587
}

backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala

+1-2
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616
*/
1717
package org.apache.gluten.execution
1818

19-
import org.apache.gluten.backendsapi.clickhouse.CHIteratorApi
2019
import org.apache.gluten.extension.ValidationResult
2120
import org.apache.gluten.utils.{BroadcastHashJoinStrategy, CHJoinValidateUtil, ShuffleHashJoinStrategy}
2221

@@ -75,7 +74,7 @@ case class CHBroadcastBuildSideRDD(
7574

7675
override def genBroadcastBuildSideIterator(): Iterator[ColumnarBatch] = {
7776
CHBroadcastBuildSideCache.getOrBuildBroadcastHashTable(broadcasted, broadcastContext)
78-
CHIteratorApi.genCloseableColumnBatchIterator(Iterator.empty)
77+
Iterator.empty
7978
}
8079
}
8180

0 commit comments

Comments
 (0)