jinchengchenghh commented on code in PR #9074: URL: https://github.com/apache/incubator-gluten/pull/9074#discussion_r2005889396
########## backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarCollectTailExec.scala: ########## @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} +import org.apache.gluten.extension.columnar.transition.Convention + +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.execution.{ShuffledColumnarBatchRDD, SparkPlan} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleWriteMetricsReporter} +import org.apache.spark.sql.metric.SQLColumnarShuffleReadMetricsReporter +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class ColumnarCollectTailExec( + limit: Int, + child: SparkPlan +) extends ColumnarCollectTailBaseExec(limit, child) { + + override def batchType(): Convention.BatchType = + BackendsApiManager.getSettings.primaryBatchType + + private def collectTailRows( + partitionIter: Iterator[ColumnarBatch], + limit: Int): Iterator[ColumnarBatch] = { + if (partitionIter.isEmpty) { + return Iterator.empty + } + + val result = new Iterator[ColumnarBatch] { + private var rowsCollected = 0 + + override def hasNext: Boolean = rowsCollected < limit && partitionIter.hasNext + + override def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException("No more batches available.") + } + + val currentBatch = partitionIter.next() + val currentBatchRowCount = currentBatch.numRows() + val remaining = limit - rowsCollected + Review Comment: What if first batch has 10 rows, second batch has 5 rows, we need tail 6 rows? ########## gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCollectTailBaseExec.scala: ########## @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.extension.ValidationResult + +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} +import org.apache.spark.sql.execution.{LimitExec, SparkPlan} + +abstract class ColumnarCollectTailBaseExec( + limit: Int, + childPlan: SparkPlan +) extends LimitExec + with ValidatablePlan { + + override def outputPartitioning: Partitioning = SinglePartition + + override protected def doValidateInternal(): ValidationResult = { + val isSupported = BackendsApiManager.getSettings.supportCollectTailExec() Review Comment: Don't need to validate again, duplicated with`FallbackByBackendSettings` , may also refactor for ColumnarCollectLimitBaseExec -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
