Closed KevinyhZou closed 1 month ago
CHNativeColumnarToRow
+- ^(2) ProjectExecTransformer [2024-08-26 AS day#0, id#12L, name#13]
+- ^(2) HashAggregateTransformer(keys=[id#12L, name#13, spark_grouping_id#11L], functions=[], isStreamingAgg=false)
+- ^(2) InputIteratorTransformer[id#12L, name#13, spark_grouping_id#11L]
+- ColumnarExchange hashpartitioning(id#12L, name#13, spark_grouping_id#11L, 1), ENSURE_REQUIREMENTS, [plan_id=112], [shuffle_writer_type=hash], [OUTPUT] List(id:LongType, name:StringType, spark_grouping_id:LongType)
+- ^(1) HashAggregateTransformer(keys=[id#12L, name#13, spark_grouping_id#11L], functions=[], isStreamingAgg=false)
+- ^(1) FilterExecTransformer (isnotnull(name#13) AND (name#13 = a124))
+- ^(1) ExpandExecTransformer [[id#6L, null, 1], [id#6L, name#7, 0]], [id#12L, name#13, spark_grouping_id#11L]
+- ^(1) ProjectExecTransformer [id#6L, name#7]
+- ^(1) NativeFileScan parquet default.test_tbl2[id#6L,name#7,day#8] Batched: true, DataFilters: [], Format: Parquet, Location: CatalogFileIndex(1 paths)[hdfs://testcluster/user/hive/warehouse/test_tbl2], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:bigint,name:string>
分别需要针对[id#6L, null, 1], [id#6L, name#7, 0]], [id#12L, name#13, spark_grouping_id#11L] 三组fields 进行expand,按照ExpandTransform
逻辑,首先对[id#6L, null, 1]执行expand,数据到下游后,被filter where name = 'a124'
, 无行数输出,导致ISource中误认为数据已经结束
从而导致提前结束expand,其他两组fields 的expand 未执行,从而导致无结果输出。
奇怪,filter过滤后block为空,按理不应该会往下游传递一个空的block的。该情况下,应该继续消费下一个block,直到input port是finished状态。
isource消费到一个空的block不正常。
output port 被设置为finished了 ,导致expandTransform 这里
input 也被close掉了
不明白。 filter 对一个block过滤后,结果是空的,这个应该是常见的情况。这也不会造成它会给下游算子一个空的block
谁的output port变成finished了?怎么被设置为fihished了?
The filter transform enters into following branch, it seems unreasonable
Should materialize the const null column
正常情况下,如果某一列是const,则会认为起后续block的对应列也都是相同值的const列。但在expand算子,这个默认被打破了。出现部分block的该位置列是const,其他的却不是。应该消除该列为const的情况。
Backend
CH (ClickHouse)
Bug description
do the query
the valina result is
the gluten result is empty.
Spark version
Spark-3.3.x
Spark configurations
No response
System information
No response
Relevant logs
No response