Skip to content

[GLUTEN-7600][VL] Remove EmptySchemaWorkaround#7620

Merged
zhztheplayer merged 9 commits intoapache:mainfrom
zhztheplayer:wip-7600-2
Oct 22, 2024
Merged

[GLUTEN-7600][VL] Remove EmptySchemaWorkaround#7620
zhztheplayer merged 9 commits intoapache:mainfrom
zhztheplayer:wip-7600-2

Conversation

@zhztheplayer
Copy link
Copy Markdown
Member

@zhztheplayer zhztheplayer commented Oct 21, 2024

Part of #7600

Remove rules EmptySchemaWorkaround.PlanOneRowRelation / EmptySchemaWorkaround.FallbackEmptySchemaRelation / OffloadHashAggregate. Inline some workaround logics into utility class ColumnarBatches or into operator validation procedures.

@github-actions github-actions bot added the VELOX label Oct 21, 2024
@github-actions
Copy link
Copy Markdown

#7600

@github-actions github-actions bot added the CORE works for Gluten Core label Oct 21, 2024
@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

checkNullTypeRepartition(
spark.table("lineitem").selectExpr("null as x", "null as y").repartition(),
1
0
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Query plan changed from

   VeloxColumnarToRow
   +- ShuffleQueryStage 0
      +- ColumnarExchange SinglePartition, REBALANCE_PARTITIONS_BY_NONE, [plan_id=979], [shuffle_writer_type=hash], [OUTPUT] ArraySeq(x:NullType, y:NullType), [OUTPUT] ArraySeq(x:NullType, y:NullType)
         +- VeloxResizeBatches 1024, 2147483647
            +- RowToVeloxColumnar
               +- *(1) Project [null AS x#296, null AS y#297]
                  +- *(1) ColumnarToRow
                     +- FileScan parquet [] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/opt/gluten/backends-velox/target/scala-2.13/test-classes/tpch-da..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<>

to

   VeloxColumnarToRow
   +- ShuffleQueryStage 0
      +- ColumnarExchange SinglePartition, REBALANCE_PARTITIONS_BY_NONE, [plan_id=957], [shuffle_writer_type=hash], [OUTPUT] ArraySeq(x:NullType, y:NullType), [OUTPUT] ArraySeq(x:NullType, y:NullType)
         +- VeloxResizeBatches 1024, 2147483647
            +- ^(12) ProjectExecTransformer [null AS x#296, null AS y#297]
               +- ^(12) NativeFileScan parquet [] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex(1 paths)[file:/opt/gluten/backends-velox/target/scala-2.13/test-classes/tpch-da..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct<>

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

Sig[CheckOverflow](CHECK_OVERFLOW),
Sig[MakeDecimal](MAKE_DECIMAL),
Sig[PromotePrecision](PROMOTE_PRECISION),
Sig[MonotonicallyIncreasingID](MONOTONICALLY_INCREASING_ID),
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Due to #7628

Comment on lines -213 to -219
if ((list == null || list.isEmpty) && childCtx != null) {
// The computing for this project is not needed.
// the child may be an input adapter and childCtx is null. In this case we want to
// make a read node with non-empty base_schema.
context.registerEmptyRelToOperator(operatorId)
return childCtx
}
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

context.registerEmptyRelToOperator(operatorId) looks to be suitable only when the operator simply outputs all the inputs it receives. Which is not the case here for list.isEmpty

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

Comment on lines +94 to +97
if (child.output.isEmpty) {
// See: https://github.com/apache/incubator-gluten/issues/7600.
return Some("Shuffle with empty schema is not supported")
}
Copy link
Copy Markdown
Member Author

@zhztheplayer zhztheplayer Oct 22, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Empty schema batches should pass through shuffle so that they can be handled in reducer-side operators within the row number information they carried on.

This is a temporary code path to disable shuffle for empty schema input. We should finally remove it to add the support.

cc @marin-ma

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Got it. Thanks!

@zhztheplayer zhztheplayer marked this pull request as ready for review October 22, 2024 05:47

private static BatchType identifyBatchType(ColumnarBatch batch) {
if (batch.numCols() == 0) {
// zero-column batch considered as heavy batch
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

comment should be updated

Copy link
Copy Markdown
Contributor

@marin-ma marin-ma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks!

Comment on lines +903 to +904
.exclude(
"SPARK-16633: lead/lag should return the default value if the offset row does not exist")
Copy link
Copy Markdown
Member Author

@zhztheplayer zhztheplayer Oct 22, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@github-actions
Copy link
Copy Markdown

Run Gluten Clickhouse CI

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

CLICKHOUSE CORE works for Gluten Core VELOX

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants