diff --git a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala index 51e8174da7fb..40a2c15b8250 100644 --- a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala +++ b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/BackendTestSettings.scala @@ -98,6 +98,13 @@ abstract class BackendTestSettings { exclusion.add(Exclude(testNames: _*)) this } + def includeCH(testNames: String*): SuiteSettings = { + this + } + def excludeCH(testNames: String*): SuiteSettings = { + exclude(testNames: _*) + this + } def includeByPrefix(prefixes: String*): SuiteSettings = { inclusion.add(IncludeByPrefix(prefixes: _*)) this diff --git a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SQLQueryTestSettings.scala b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SQLQueryTestSettings.scala index 36d21e899313..3c2da2b1f893 100644 --- a/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SQLQueryTestSettings.scala +++ b/gluten-ut/common/src/test/scala/org/apache/gluten/utils/SQLQueryTestSettings.scala @@ -20,4 +20,6 @@ trait SQLQueryTestSettings { def getSupportedSQLQueryTests: Set[String] def getOverwriteSQLQueryTests: Set[String] + + def getIgnoredSQLQueryTests: List[String] = List.empty } diff --git a/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala index d20a419597d1..5b84888376aa 100644 --- a/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala +++ b/gluten-ut/spark35/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala @@ -20,9 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.execution.datasources.parquet.GlutenParquetV1FilterSuite -/** testing use_local_format parquet reader. - * FIXME: Run this suite in Spark 35 CI Pipeline - * */ +/** testing use_local_format parquet reader. **/ class GlutenParquetV1FilterSuite2 extends GlutenParquetV1FilterSuite { override def sparkConf: SparkConf = super.sparkConf diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala index 5d420ff19f89..eb12d8c7a5f6 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala @@ -19,7 +19,247 @@ package org.apache.gluten.utils.clickhouse import org.apache.gluten.utils.SQLQueryTestSettings object ClickHouseSQLQueryTestSettings extends SQLQueryTestSettings { - override def getSupportedSQLQueryTests: Set[String] = Set() + override def getSupportedSQLQueryTests: Set[String] = SUPPORTED_SQL_QUERY_LIST - override def getOverwriteSQLQueryTests: Set[String] = Set() + override def getOverwriteSQLQueryTests: Set[String] = OVERWRITE_SQL_QUERY_LIST + + override def getIgnoredSQLQueryTests: List[String] = IGNORE_SQL_QUERY_LIST + + // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list + val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "bitwise.sql", + "cast.sql", + "change-column.sql", + // CH- "charvarchar.sql", + "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", + "comments.sql", + "comparator.sql", + "count.sql", + "cross-join.sql", + "csv-functions.sql", + // CH- "cte-legacy.sql", + "cte-nested.sql", + // CH- "cte-nonlegacy.sql", + // CH- "cte.sql", + "current_database_catalog.sql", + "date.sql", + "datetime-formatting-invalid.sql", + // Velox had different handling for some illegal cases. + // "datetime-formatting-legacy.sql", + // "datetime-formatting.sql", + "datetime-legacy.sql", + "datetime-parsing-invalid.sql", + "datetime-parsing-legacy.sql", + "datetime-parsing.sql", + "datetime-special.sql", + // CH - "decimalArithmeticOperations.sql", + // "describe-part-after-analyze.sql", + "describe-query.sql", + "describe-table-after-alter-table.sql", + "describe-table-column.sql", + "describe.sql", + "except-all.sql", + "except.sql", + "extract.sql", + "group-by-filter.sql", + // CH - "group-by-ordinal.sql", + "grouping_set.sql", + "having.sql", + "ignored.sql", + "ilike-all.sql", + "ilike-any.sql", + "inline-table.sql", + "inner-join.sql", + "intersect-all.sql", + "interval.sql", + "join-empty-relation.sql", + // CH - "join-lateral.sql", + // CH - "json-functions.sql", + "like-all.sql", + "like-any.sql", + "limit.sql", + "literals.sql", + "map.sql", + // CH- "misc-functions.sql", + "natural-join.sql", + "null-handling.sql", + // CH- "null-propagation.sql", + "operators.sql", + "order-by-nulls-ordering.sql", + "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", + "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", + "query_regex_column.sql", + // CH- "random.sql", + // CH - "regexp-functions.sql", + "show-create-table.sql", + "show-tables.sql", + "show-tblproperties.sql", + "show-views.sql", + "show_columns.sql", + "sql-compatibility-functions.sql", + "string-functions.sql", + "struct.sql", + // CH - "subexp-elimination.sql", + "table-aliases.sql", + // CH -"table-valued-functions.sql", + "tablesample-negative.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-within-and-or.sql", + "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-order-by.sql", + // CH- "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", + // CH -"subquery/negative-cases/invalid-correlation.sql", + "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/subquery-in-from.sql", + "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", + "postgreSQL/aggregates_part3.sql", + "postgreSQL/aggregates_part4.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", + "postgreSQL/create_view.sql", + "postgreSQL/date.sql", + "postgreSQL/float4.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/interval.sql", + "postgreSQL/join.sql", + "postgreSQL/limit.sql", + "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", + "postgreSQL/select_having.sql", + "postgreSQL/select_implicit.sql", + "postgreSQL/strings.sql", + "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", + "postgreSQL/union.sql", + "postgreSQL/window_part1.sql", + "postgreSQL/window_part2.sql", + "postgreSQL/window_part3.sql", + "postgreSQL/window_part4.sql", + "postgreSQL/with.sql", + "datetime-special.sql", + "timestamp-ansi.sql", + "timestamp.sql", + "arrayJoin.sql", + "binaryComparison.sql", + "booleanEquality.sql", + "caseWhenCoercion.sql", + "concat.sql", + "dateTimeOperations.sql", + "decimalPrecision.sql", + "division.sql", + "elt.sql", + "ifCoercion.sql", + "implicitTypeCasts.sql", + "inConversion.sql", + "mapZipWith.sql", + "promoteStrings.sql", + "stringCastAndExpressions.sql", + "widenSetOperationTypes.sql", + "windowFrameCoercion.sql", + "timestamp-ltz.sql", + "timestamp-ntz.sql", + "timezone.sql", + // CH- "transform.sql", + "try_arithmetic.sql", + "try_cast.sql", + "udaf.sql", + "union.sql", + "using-join.sql", + "window.sql", + "udf-union.sql", + "udf-window.sql", + "ansi/cast.sql", + "ansi/decimalArithmeticOperations.sql", + "ansi/map.sql", + "ansi/datetime-parsing-invalid.sql", + "ansi/string-functions.sql", + "ansi/interval.sql", + "ansi/date.sql", + "ansi/timestamp.sql", + "ansi/try_arithmetic.sql", + "ansi/literals.sql", + "timestampNTZ/timestamp-ansi.sql", + "timestampNTZ/timestamp.sql", + "udf/udf-intersect-all.sql - Scala UDF", + "udf/udf-except-all.sql - Scala UDF", + "udf/udf-udaf.sql - Scala UDF", + "udf/udf-except.sql - Scala UDF", + "udf/udf-pivot.sql - Scala UDF", + "udf/udf-inline-table.sql - Scala UDF", + "udf/postgreSQL/udf-select_having.sql - Scala UDF", + "typeCoercion/native/decimalPrecision.sql", + "typeCoercion/native/ifCoercion.sql", + "typeCoercion/native/dateTimeOperations.sql", + "typeCoercion/native/booleanEquality.sql", + "typeCoercion/native/mapZipWith.sql", + "typeCoercion/native/caseWhenCoercion.sql", + "typeCoercion/native/widenSetOperationTypes.sql", + "typeCoercion/native/stringCastAndExpressions.sql", + "typeCoercion/native/inConversion.sql", + "typeCoercion/native/division.sql", + "typeCoercion/native/mapconcat.sql" + ) + + val OVERWRITE_SQL_QUERY_LIST: Set[String] = Set( + // The calculation formulas for corr, skewness, kurtosis, variance, and stddev in Velox differ + // slightly from those in Spark, resulting in some differences in the final results. + // Overwrite below test cases. + // -- SPARK-24369 multiple distinct aggregations having the same argument set + // -- Aggregate with nulls. + "group-by.sql", + "udf/udf-group-by.sql" + // Overwrite some results of regr_intercept, regr_r2, corr. + // CH - "linear-regression.sql" + ) + + val IGNORE_SQL_QUERY_LIST: List[String] = List( + "udf/udf-count.sql - Regular Python UDF", + "udf/udf-except.sql - Regular Python UDF", + "udf/udf-except-all.sql - Regular Python UDF", + "udf/udf-natural-join.sql - Regular Python UDF", + "udf/udf-outer-join.sql - Regular Python UDF", + "udf/udf-pivot.sql - Regular Python UDF", + "udf/udf-intersect-all.sql - Regular Python UDF", + "udf/udf-union.sql - Regular Python UDF", + "udf/udf-having.sql - Regular Python UDF", + "udf/udf-group-analytics.sql - Regular Python UDF", + "udf/udf-group-by.sql - Regular Python UDF", + // CH excludes following + "typeCoercion/native/windowFrameCoercion.sql", + "typeCoercion/native/promoteStrings.sql", + "typeCoercion/native/concat.sql" + ) } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index e9cb3aea85c7..3c22b6bcd598 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -18,1705 +18,2148 @@ package org.apache.gluten.utils.clickhouse import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} +import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.connector._ -import org.apache.spark.sql.errors._ +import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.clickhouse.ClickHouseAdaptiveQueryExecSuite import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite -import org.apache.spark.sql.execution.datasources.exchange._ -import org.apache.spark.sql.execution.datasources.json._ +import org.apache.spark.sql.execution.datasources.csv.{GlutenCSVLegacyTimeParserSuite, GlutenCSVv1Suite, GlutenCSVv2Suite} +import org.apache.spark.sql.execution.datasources.exchange.GlutenValidateRequirementsSuite +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} import org.apache.spark.sql.execution.datasources.orc._ import org.apache.spark.sql.execution.datasources.parquet._ import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} import org.apache.spark.sql.execution.exchange.GlutenEnsureRequirementsSuite -import org.apache.spark.sql.execution.joins.{GlutenBroadcastJoinSuite, GlutenExistenceJoinSuite, GlutenInnerJoinSuiteForceShjOn, GlutenOuterJoinSuiteForceShjOn} -import org.apache.spark.sql.extension.{GlutenCustomerExtensionSuite, GlutenSessionExtensionSuite} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenCustomerExtensionSuite, GlutenSessionExtensionSuite} import org.apache.spark.sql.gluten.GlutenFallbackSuite import org.apache.spark.sql.hive.execution.GlutenHiveSQLQueryCHSuite import org.apache.spark.sql.sources._ -import org.apache.spark.sql.statistics.SparkFunctionStatistics // Some settings' line length exceeds 100 // scalastyle:off line.size.limit class ClickHouseTestSettings extends BackendTestSettings { - // disable tests that will break the whole UT - override def shouldRun(suiteName: String, testName: String): Boolean = { - val preCheck = suiteName.split("[.]").last match { - case "GlutenCSVSuite" => !csvCoreDumpCases.contains(testName) - case "GlutenCSVv1Suite" => !csvCoreDumpCases.contains(testName) - case "GlutenCSVv2Suite" => !csvCoreDumpCases.contains(testName) - case "GlutenCSVLegacyTimeParserSuite" => !csvCoreDumpCases.contains(testName) - case "GlutenDataFrameSuite" => !dfCoreDumpCases.contains(testName) - case "GlutenDatasetSuite" => !dsSlowCases.contains(testName) - case "GlutenSQLQuerySuite" => !sqlQuerySlowCases.contains(testName) - // Below 2 suites temporarily ignored because of gluten domain name change - case "GlutenClickHouseMergeTreeWriteOnHDFSSuite" => - false - case "GlutenClickHouseMergeTreeWriteOnS3Suite" => - false - case "GlutenDataFrameWriterV2Suite" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenDataSourceV2DataFrameSuite" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenDataSourceV2FunctionSuite" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenDataSourceV2SQLSuiteV1Filter" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenDataSourceV2SQLSuiteV2Filter" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenMetadataColumnSuite" => false // nativeDoValidate failed due to spark conf cleanup - case "GlutenQueryCompilationErrorsDSv2Suite" => - false // nativeDoValidate failed due to spark conf cleanup - case "GlutenBloomFilterAggregateQuerySuite" => - !bloomFilterCases.contains(testName) - case "GlutenSortShuffleSuite" => false - case _ => true - } - preCheck && super.shouldRun(suiteName, testName) - } - - private val csvCoreDumpCases: Seq[String] = Seq( - "test with alternative delimiter and quote", - "SPARK-24540: test with multiple character delimiter (comma space)", - "DDL test with tab separated file", - "test with null quote character", - "SPARK-24540: test with multiple (crazy) character delimiter", - "nullable fields with user defined null value of \"null\"", - "SPARK-15585 turn off quotations", - "SPARK-29101 test count with DROPMALFORMED mode" - ) - - private val dfCoreDumpCases: Seq[String] = Seq( - "repartitionByRange", - GLUTEN_TEST + "repartitionByRange" - ) - - private val dsSlowCases: Seq[String] = Seq( - "SPARK-16995: flat mapping on Dataset containing a column created with lit/expr" - ) - - private val sqlQuerySlowCases: Seq[String] = Seq( - "SPARK-33084: Add jar support Ivy URI in SQL" - ) - - private val bloomFilterCases: Seq[String] = Seq( - // Currently return a empty set(same reason as sum(empty set), - // both behaviors are acceptable. - "Test that bloom_filter_agg produces a NULL with empty input" - ) - - enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite].exclude( - "test ApproxCountDistinctForIntervals with large number of endpoints") - enableSuite[GlutenApproximatePercentileQuerySuite].exclude( - "SPARK-32908: maximum target error in percentile_approx") + enableSuite[ClickHouseAdaptiveQueryExecSuite] + .includeAllGlutenTests() + .includeByPrefix( + "SPARK-29906", + "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-31658", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", + "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[FallbackStrategiesSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + .excludeCH("test ApproxCountDistinctForIntervals with large number of endpoints") + enableSuite[GlutenApproximatePercentileQuerySuite] + // requires resource files from Vanilla spark jar + .exclude("SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("SPARK-45786: Decimal multiply, divide, remainder, quot") + .excludeCH("% (Remainder)") + .excludeCH("SPARK-17617: % (Remainder) double % double on super big double") + .excludeCH("pmod") + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .includeCH("column pruning - non-readable file") + enableSuite[GlutenBitmapExpressionsQuerySuite] + enableSuite[GlutenBitwiseExpressionsSuite] enableSuite[GlutenBloomFilterAggregateQuerySuite] - .exclude("Test bloom_filter_agg and might_contain") - .excludeGlutenTest("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBloomFilterAggregateQuerySuiteCGOff] + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBroadcastExchangeSuite] + enableSuite[GlutenBroadcastJoinSuite] + .includeCH("Shouldn't change broadcast join buildSide if user clearly specified") + .includeCH("Shouldn't bias towards build right if user didn't specify") + .includeCH("SPARK-23192: broadcast hint should be retained after using the cached data") + .includeCH("broadcast join where streamed side's output partitioning is HashPartitioning") + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("disable bucketing when the output doesn't contain all bucketing columns") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + .includeCH("write bucketed data") + .includeCH("write bucketed data with sortBy") + .includeCH("write bucketed data without partitionBy") + .includeCH("write bucketed data without partitionBy with sortBy") + .includeCH("write bucketed data with bucketing disabled") + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenCSVLegacyTimeParserSuite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenCSVv1Suite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39469: Infer schema for columns with all dates") + .excludeCH("SPARK-40474: Infer schema for columns with a mix of dates and timestamp") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + enableSuite[GlutenCSVv2Suite] + .exclude("Gluten - test for FAILFAST parsing mode") + // Rule org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown in batch + // Early Filter and Projection Push-Down generated an invalid plan + .exclude("SPARK-26208: write and read empty data to csv file with headers") + // file cars.csv include null string, Arrow not support to read + .exclude("old csv data source name works") + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") enableSuite[GlutenCTEHintSuite] enableSuite[GlutenCTEInlineSuiteAEOff] enableSuite[GlutenCTEInlineSuiteAEOn] enableSuite[GlutenCachedTableSuite] - .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") - .exclude("analyzes column statistics in cached query") - .excludeGlutenTest("InMemoryRelation statistics") + .exclude("A cached table preserves the partitioning and ordering of its cached SparkPlan") + .includeCH("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .includeCH("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + .excludeCH("Gluten - InMemoryRelation statistics") + enableSuite[GlutenCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("Gluten - data type casting") + enableSuite[GlutenCoalesceShufflePartitionsSuite] + .excludeByPrefix("determining the number of reducers") + .excludeCH("SPARK-46590 adaptive query execution works correctly with broadcast join and union") + .excludeCH("SPARK-46590 adaptive query execution works correctly with cartesian join and union") + .excludeCH("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") + .excludeCH("Do not reduce the number of shuffle partition for repartition") + .excludeCH("Union two datasets with different pre-shuffle partition number") + .excludeCH("SPARK-34790: enable IO encryption in AQE partition coalescing") + .excludeCH("Gluten - determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: aggregate operator") + .excludeCH("Gluten - determining the number of reducers: join operator") + .excludeCH("Gluten - determining the number of reducers: complex query 1") + .excludeCH("Gluten - determining the number of reducers: complex query 2") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned") + enableSuite[GlutenCollapseProjectExecTransformerSuite] + .excludeCH("Gluten - Support ProjectExecTransformer collapse") + enableSuite[GlutenCollectionExpressionsSuite] + // Rewrite in Gluten to replace Seq with Array + .exclude("Shuffle") + .excludeGlutenTest("Shuffle") + .excludeCH("Sequence of numbers") + .excludeCH("Array Insert") + .excludeCH("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") + .excludeCH( + "SPARK-36740: ArrayMin/ArrayMax/SortArray should handle NaN greater than non-NaN value") + .excludeCH("SPARK-42401: Array insert of null value (explicit)") + .excludeCH("SPARK-42401: Array insert of null value (implicit)") enableSuite[GlutenColumnExpressionSuite] - .exclude("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") - .exclude("withField should add field with no name") - .exclude("withField should add field to nullable struct") - .exclude("withField should add field to nested nullable struct") - .exclude("withField should add multiple fields to nullable struct") - .exclude("withField should add multiple fields to nested nullable struct") - .exclude("withField should replace field in nullable struct") - .exclude("withField should replace field in nested nullable struct") - .exclude("withField should replace multiple fields in nullable struct") - .exclude("withField should replace multiple fields in nested nullable struct") - .exclude("withField should replace all fields with given name in struct") - .exclude("withField user-facing examples") - .exclude("dropFields should drop field in nullable struct") - .exclude("dropFields should drop field with no name in struct") - .exclude("dropFields should drop field in nested nullable struct") - .exclude("dropFields should drop multiple fields in nested nullable struct") - .exclude("dropFields should drop all fields with given name in struct") - .exclude("dropFields user-facing examples") - .exclude("should move field up one level of nesting") - .exclude("SPARK-36778: add ilike API for scala") + // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. + // The final caught Spark exception's getCause().getMessage() contains 'errMsg' but does not + // equal 'errMsg' exactly. The following two tests will be skipped and overridden in Gluten. + .includeCH("raise_error") + .includeCH("assert_true") + .excludeCH("withField should add field with no name") + .excludeCH("withField should replace all fields with given name in struct") + .excludeCH("withField user-facing examples") + .excludeCH("dropFields should drop field with no name in struct") + .excludeCH("dropFields should drop all fields with given name in struct") + enableSuite[GlutenComplexTypeSuite] enableSuite[GlutenComplexTypesSuite] - enableSuite[GlutenConfigBehaviorSuite].exclude( - "SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + enableSuite[GlutenConditionalExpressionSuite] + .excludeCH("case when") + enableSuite[GlutenConfigBehaviorSuite] + // Will be fixed by cleaning up ColumnarShuffleExchangeExec. + .exclude("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCreateTableAsSelectSuite] + .includeCH("CREATE TABLE USING AS SELECT based on the file without write permission") + .includeCH("create a table, drop it and create another one with the same name") enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCustomerExtensionSuite] + enableSuite[GlutenDDLSourceLoadSuite] enableSuite[GlutenDSV2CharVarcharTestSuite] enableSuite[GlutenDSV2SQLInsertTestSuite] enableSuite[GlutenDataFrameAggregateSuite] - .exclude("average") - .exclude("zero average") - .exclude("zero stddev") - .exclude("collect functions") - .exclude("collect functions structs") - .exclude("SPARK-17641: collect functions should not collect null values") - .exclude("collect functions should be able to cast to array type with no null values") - .exclude("SPARK-17616: distinct aggregate combined with a non-partial aggregate") - .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it") - .excludeGlutenTest("SPARK-19471: AggregationIterator does not initialize the generated" + - " result projection before using it") - .exclude("SPARK-26021: NaN and -0.0 in grouping expressions") - .exclude("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") - .exclude("SPARK-32136: NormalizeFloatingNumbers should work on null struct") - .exclude("SPARK-34713: group by CreateStruct with ExtractValue") - .exclude("SPARK-34716: Support ANSI SQL intervals by the aggregate function `sum`") - .exclude("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") - .exclude("SPARK-35412: groupBy of year-month/day-time intervals should work") - .exclude("SPARK-36926: decimal average mistakenly overflow") - .excludeGlutenTest("use gluten hash agg to replace vanilla spark sort agg") + // Test for vanilla spark codegen, not apply for Gluten + .exclude("SPARK-43876: Enable fast hashmap for distinct queries") + .exclude( + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // incorrect result, distinct NaN case + "SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", + // Replaced with another test. + "SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it" + ) + .includeCH( + "zero moments", // [velox does not return NaN] + // Velox's collect_list / collect_set are by design declarative aggregate so plan check + // for ObjectHashAggregateExec will fail. + "SPARK-22223: ObjectHashAggregate should not introduce unnecessary shuffle", + "SPARK-31620: agg with subquery (whole-stage-codegen = true)", + "SPARK-31620: agg with subquery (whole-stage-codegen = false)" + ) + .excludeCH("linear regression") + .excludeCH("collect functions") + .excludeCH("collect functions structs") + .excludeCH("SPARK-17641: collect functions should not collect null values") + .excludeCH("collect functions should be able to cast to array type with no null values") + .excludeCH("SPARK-45599: Neither 0.0 nor -0.0 should be dropped when computing percentile") + .excludeCH("SPARK-34716: Support ANSI SQL intervals by the aggregate function `sum`") + .excludeCH("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") + .excludeCH("SPARK-35412: groupBy of year-month/day-time intervals should work") + .excludeCH("SPARK-36054: Support group by TimestampNTZ column") enableSuite[GlutenDataFrameAsOfJoinSuite] enableSuite[GlutenDataFrameComplexTypeSuite] enableSuite[GlutenDataFrameFunctionsSuite] - .exclude("map with arrays") - .exclude("flatten function") - .exclude("aggregate function - array for primitive type not containing null") + // blocked by Velox-5768 .exclude("aggregate function - array for primitive type containing null") .exclude("aggregate function - array for non-primitive type") - .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") - .exclude("SPARK-24734: Fix containsNull of Concat for array type") - .exclude("transform keys function - primitive data types") - .exclude("transform keys function - Invalid lambda functions and exceptions") - .exclude("transform values function - test primitive data types") - .exclude("transform values function - test empty") + // Rewrite this test because Velox sorts rows by key for primitive data types, which disrupts the original row sequence. + .includeCH("map_zip_with function - map of primitive types") + .excludeCH("map with arrays") + .excludeCH("flatten function") + .excludeCH("SPARK-41233: array prepend") + .excludeCH("array_insert functions") + .excludeCH("aggregate function - array for primitive type not containing null") + .excludeCH("transform keys function - primitive data types") + .excludeCH("transform values function - test primitive data types") + .excludeCH("transform values function - test empty") + .excludeCH("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") + .excludeCH("mask function") enableSuite[GlutenDataFrameHintSuite] enableSuite[GlutenDataFrameImplicitsSuite] - enableSuite[GlutenDataFrameJoinSuite].exclude( - "SPARK-32693: Compare two dataframes with same schema except nullable property") + enableSuite[GlutenDataFrameJoinSuite] + .excludeCH("SPARK-32693: Compare two dataframes with same schema except nullable property") enableSuite[GlutenDataFrameNaFunctionsSuite] - .exclude("replace nan with float") - .exclude("replace nan with double") + .includeCH( + // NaN case + "replace nan with float", + "replace nan with double" + ) enableSuite[GlutenDataFramePivotSuite] - .exclude("pivot with column definition in groupby") - .exclude("pivot with timestamp and count should not print internal representation") - .exclude("SPARK-38133: Grouping by TIMESTAMP_NTZ should not corrupt results") + // substring issue + .includeCH("pivot with column definition in groupby") + // array comparison not supported for values that contain nulls + .includeCH( + "pivot with null and aggregate type not supported by PivotFirst returns correct result") + .excludeCH("SPARK-38133: Grouping by TIMESTAMP_NTZ should not corrupt results") enableSuite[GlutenDataFrameRangeSuite] + .excludeCH("SPARK-20430 Initialize Range parameters in a driver side") + .excludeByPrefix("Cancelling stage in a query with Range") enableSuite[GlutenDataFrameSelfJoinSuite] enableSuite[GlutenDataFrameSessionWindowingSuite] - .exclude("simple session window with record at window start") - .exclude("session window groupBy statement") - .exclude("SPARK-36465: filter out events with negative/zero gap duration") - .exclude("session window groupBy with multiple keys statement") - .exclude("session window groupBy with multiple keys statement - one distinct") - .exclude("session window groupBy with multiple keys statement - two distinct") - .exclude("session window groupBy with multiple keys statement - keys overlapped with sessions") - .exclude("session window with multi-column projection") - .exclude("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow") + .excludeCH("simple session window with record at window start") + .excludeCH("session window groupBy statement") + .excludeCH("session window groupBy with multiple keys statement") + .excludeCH("session window groupBy with multiple keys statement - two distinct") + .excludeCH( + "session window groupBy with multiple keys statement - keys overlapped with sessions") + .excludeCH("SPARK-36465: filter out events with negative/zero gap duration") + .excludeCH("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow") enableSuite[GlutenDataFrameSetOperationsSuite] - .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") - .exclude( - "SPARK-34283: SQL-style union using Dataset, keep necessary deduplicate in multiple unions") - .exclude("union should union DataFrames with UDTs (SPARK-13410)") - .exclude( - "SPARK-32376: Make unionByName null-filling behavior work with struct columns - simple") - .exclude( - "SPARK-32376: Make unionByName null-filling behavior work with struct columns - nested") - .exclude("SPARK-32376: Make unionByName null-filling behavior work with struct columns - case-sensitive cases") - .exclude( - "SPARK-32376: Make unionByName null-filling behavior work with struct columns - edge case") - .exclude("SPARK-35290: Make unionByName null-filling behavior work with struct columns - sorting edge case") - .exclude( - "SPARK-32376: Make unionByName null-filling behavior work with struct columns - deep expr") - .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") - .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") - .exclude( - "SPARK-36673: Only merge nullability for Unions of struct" - ) // disabled due to case-insensitive not supported in CH tuple + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + .excludeCH("union should union DataFrames with UDTs (SPARK-13410)") + .excludeCH( + "SPARK-35756: unionByName support struct having same col names but different sequence") + .excludeCH("SPARK-36673: Only merge nullability for Unions of struct") + .excludeCH("SPARK-36797: Union should resolve nested columns as top-level columns") enableSuite[GlutenDataFrameStatSuite] + .excludeCH("SPARK-30532 stat functions to understand fully-qualified column name") + .excludeCH("special crosstab elements (., '', null, ``)") enableSuite[GlutenDataFrameSuite] - .exclude("Uuid expressions should produce same results at retries in the same DataFrame") - .exclude("SPARK-28224: Aggregate sum big decimal overflow") - .exclude("SPARK-28067: Aggregate sum should not return wrong results for decimal overflow") - .exclude("SPARK-35955: Aggregate avg should not return wrong results for decimal overflow") - .exclude("describe") - .exclude("SPARK-34165: Add count_distinct to summary") - .exclude("getRows: array") - .exclude("showString: array") - .exclude("showString: array, vertical = true") - .exclude("SPARK-23023 Cast rows to strings in showString") - .exclude("SPARK-18350 show with session local timezone") - .exclude("SPARK-18350 show with session local timezone, vertical = true") - .exclude("SPARK-6899: type should match when using codegen") - .exclude("SPARK-7324 dropDuplicates") - .exclude( + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // decimal failed ut. + "SPARK-22271: mean overflows and returns null for some decimal variables", + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + "SPARK-9083: sort with non-deterministic expressions" + ) + .includeCH( + // Mismatch when max NaN and infinite value + "NaN is greater than all other non-NaN numeric values", + "distributeBy and localSort" + ) + // test for sort node not present but gluten uses shuffle hash join + .exclude("SPARK-41048: Improve output partitioning and ordering with AQE cache") + // Rewrite this test since it checks the physical operator which is changed in Gluten + .includeCH("SPARK-27439: Explain result should match collected result after view change") + .excludeCH("SPARK-28067: Aggregate sum should not return wrong results for decimal overflow") + .excludeCH("SPARK-35955: Aggregate avg should not return wrong results for decimal overflow") + .excludeCH("summary") + .excludeCH( "SPARK-8608: call `show` on local DataFrame with random columns should return same value") - .exclude("SPARK-8609: local DataFrame with random columns should return same value after sort") - .exclude("SPARK-9083: sort with non-deterministic expressions") - .exclude("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") - .exclude("distributeBy and localSort") - .exclude("reuse exchange") - .exclude("SPARK-22271: mean overflows and returns null for some decimal variables") - .exclude("SPARK-22520: support code generation for large CaseWhen") - .exclude("SPARK-24165: CaseWhen/If - nullability of nested types") - .exclude("SPARK-27671: Fix analysis exception when casting null in nested field in struct") - .exclude("summary") - .excludeGlutenTest("distributeBy and localSort") - .excludeGlutenTest("describe") - .excludeGlutenTest("Allow leading/trailing whitespace in string before casting") + .excludeCH( + "SPARK-8609: local DataFrame with random columns should return same value after sort") + .excludeCH("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") + .excludeCH("Uuid expressions should produce same results at retries in the same DataFrame") + .excludeCH("Gluten - repartitionByRange") + .excludeCH("Gluten - describe") + .excludeCH("Gluten - Allow leading/trailing whitespace in string before casting") enableSuite[GlutenDataFrameTimeWindowingSuite] - .exclude("simple tumbling window with record at window start") - .exclude("SPARK-21590: tumbling window using negative start time") - .exclude("tumbling window groupBy statement") - .exclude("tumbling window groupBy statement with startTime") - .exclude("SPARK-21590: tumbling window groupBy statement with negative startTime") - .exclude("tumbling window with multi-column projection") - .exclude("sliding window grouping") - .exclude("time window joins") - .exclude("negative timestamps") - .exclude("millisecond precision sliding windows") - enableSuite[GlutenDataFrameTungstenSuite].excludeGlutenTest("Map type with struct type as key") + .excludeCH("simple tumbling window with record at window start") + .excludeCH("SPARK-21590: tumbling window using negative start time") + .excludeCH("tumbling window groupBy statement") + .excludeCH("tumbling window groupBy statement with startTime") + .excludeCH("SPARK-21590: tumbling window groupBy statement with negative startTime") + .excludeCH("sliding window grouping") + .excludeCH("time window joins") + .excludeCH("millisecond precision sliding windows") + enableSuite[GlutenDataFrameToSchemaSuite] + .excludeCH("struct value: compatible field nullability") + .excludeCH("map value: reorder inner fields by name") + enableSuite[GlutenDataFrameTungstenSuite] enableSuite[GlutenDataFrameWindowFramesSuite] - .exclude("rows between should accept int/long values as boundary") + // Local window fixes are not added. .exclude("range between should accept int/long values as boundary") - .exclude("reverse preceding/following range between with aggregation") + .includeCH("unbounded preceding/following range between with aggregation") + .includeCH("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .excludeCH("rows between should accept int/long values as boundary") + .excludeCH("reverse preceding/following range between with aggregation") + .excludeCH( + "SPARK-41793: Incorrect result for window frames defined by a range clause on large decimals") enableSuite[GlutenDataFrameWindowFunctionsSuite] + // does not support `spark.sql.legacy.statisticalAggregate=true` (null -> NAN) .exclude("corr, covar_pop, stddev_pop functions in specific window") - .exclude( - "SPARK-13860: corr, covar_pop, stddev_pop functions in specific window LEGACY_STATISTICAL_AGGREGATE off") .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") - .exclude("SPARK-13860: covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window LEGACY_STATISTICAL_AGGREGATE off") - .exclude("lead/lag with ignoreNulls") + // does not support spill .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") .exclude("SPARK-21258: complex object in combination with spilling") + // rewrite `WindowExec -> WindowExecTransformer` .exclude( "SPARK-38237: require all cluster keys for child required distribution for window query") - .excludeGlutenTest("corr, covar_pop, stddev_pop functions in specific window") + .excludeCH("SPARK-13860: corr, covar_pop, stddev_pop functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("SPARK-13860: covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("lead/lag with ignoreNulls") + .excludeCH("SPARK-37099: Insert window group limit node for top-k computation") + .excludeCH("Gluten - corr, covar_pop, stddev_pop functions in specific window") + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + .excludeCH("view should use captured catalog and namespace for function lookup") + .excludeCH("aggregate function: lookup int average") + .excludeCH("aggregate function: lookup long average") + .excludeCH("aggregate function: lookup double average in Java") + .excludeCH("aggregate function: lookup int average w/ expression") + .excludeCH("SPARK-35390: aggregate function w/ type coercion") + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + .excludeCH("DeleteFrom: - delete with invalid predicate") + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following tests in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") enableSuite[GlutenDatasetAggregatorSuite] enableSuite[GlutenDatasetCacheSuite] enableSuite[GlutenDatasetOptimizationSuite] - .exclude("Pruned nested serializers: map of map key") - .exclude("Pruned nested serializers: map of complex key") enableSuite[GlutenDatasetPrimitiveSuite] enableSuite[GlutenDatasetSerializerRegistratorSuite] enableSuite[GlutenDatasetSuite] - .exclude("SPARK-16853: select, case class and tuple") - .exclude("select 2, primitive and tuple") - .exclude("SPARK-15550 Dataset.show() should show inner nested products as rows") - .exclude("dropDuplicates") + // Rewrite the following two tests in GlutenDatasetSuite. .exclude("dropDuplicates: columns with same column name") - .exclude("SPARK-24762: select Option[Product] field") - .exclude("SPARK-24762: typed agg on Option[Product] type") - .exclude("SPARK-26233: serializer should enforce decimal precision and scale") .exclude("groupBy.as") + .exclude("dropDuplicates") + .exclude("select 2, primitive and tuple") + .exclude("SPARK-16853: select, case class and tuple") + // TODO: SPARK-16995 may dead loop!! + .exclude("SPARK-16995: flat mapping on Dataset containing a column created with lit/expr") + .exclude("SPARK-24762: typed agg on Option[Product] type") .exclude("SPARK-40407: repartition should not result in severe data skew") .exclude("SPARK-40660: Switch to XORShiftRandom to distribute elements") + enableSuite[GlutenDatasetUnpivotSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("Hour") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("to_timestamp exception mode") + // Replaced by a gluten test to pass timezone through config. + .exclude("from_unixtime") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") + .excludeCH("DayOfYear") + .excludeCH("Quarter") + .excludeCH("Month") + .excludeCH("Day / DayOfMonth") + .excludeCH("DayOfWeek") + .excludeCH("WeekDay") + .excludeCH("WeekOfYear") + .excludeCH("add_months") + .excludeCH("months_between") + .excludeCH("TruncDate") + .excludeCH("unsupported fmt fields for trunc/date_trunc results null") + .excludeCH("to_utc_timestamp") + .excludeCH("from_utc_timestamp") + .excludeCH("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") + .excludeCH("UNIX_SECONDS") + .excludeCH("TIMESTAMP_SECONDS") enableSuite[GlutenDateFunctionsSuite] - .exclude("function to_date") + // The below two are replaced by two modified versions. .exclude("unix_timestamp") .exclude("to_unix_timestamp") + // Unsupported datetime format: specifier X is not supported by velox. .exclude("to_timestamp with microseconds precision") + // Legacy mode is not supported, assuming this mode is not commonly used. .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") - .exclude("SPARK-30766: date_trunc of old timestamps to hours and days") - .exclude("SPARK-30793: truncate timestamps before the epoch to seconds and minutes") - .excludeGlutenTest("unix_timestamp") - .excludeGlutenTest("to_unix_timestamp") - .exclude("to_utc_timestamp with column zone") - .exclude("from_utc_timestamp with column zone") + // Legacy mode is not supported and velox getTimestamp function does not throw + // exception when format is "yyyy-dd-aa". + .exclude("function to_date") + .excludeCH("SPARK-30766: date_trunc of old timestamps to hours and days") + .excludeCH("SPARK-30793: truncate timestamps before the epoch to seconds and minutes") + .excludeCH("try_to_timestamp") + .excludeCH("Gluten - to_unix_timestamp") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenDecimalPrecisionSuite] + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite] + enableSuite[GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite] + // FIXME: complex type result mismatch + .includeCH("update nested struct fields") + .includeCH("update char/varchar columns") + enableSuite[GlutenDeltaBasedUpdateTableSuite] enableSuite[GlutenDeprecatedAPISuite] - enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff].excludeGlutenTest( - "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") - enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn].excludeGlutenTest( - "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") - enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff].excludeGlutenTest( - "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") - enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn].excludeGlutenTest( - "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + .disable( + "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + .excludeGlutenTest("Subquery reuse across the whole plan") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + enableSuite[GlutenEmptyInSuite] + .excludeCH("IN with empty list") + enableSuite[GlutenEnsureRequirementsSuite] + enableSuite[GlutenExchangeSuite] + // ColumnarShuffleExchangeExec does not support doExecute() method + .exclude("shuffling UnsafeRows in exchange") + // ColumnarShuffleExchangeExec does not support SORT_BEFORE_REPARTITION + .exclude("SPARK-23207: Make repartition() generate consistent output") + // This test will re-run in GlutenExchangeSuite with shuffle partitions > 1 + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenExistenceJoinSuite] + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExternalCommandRunnerSuite] enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFallbackSuite] + .excludeCH("Gluten - test fallback event") enableSuite[GlutenFileBasedDataSourceSuite] - .exclude("SPARK-23072 Write and read back unicode column names - csv") - .excludeByPrefix("Enabling/disabling ignoreMissingFiles using") - .excludeGlutenTestsByPrefix("Enabling/disabling ignoreMissingFiles using") + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // error msg from velox is different & reader options is not supported, rewrite + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") - .exclude("SPARK-25237 compute correct input metrics in FileScanRDD") - .exclude("SPARK-30362: test input metrics for DSV2") - .exclude("SPARK-37585: test input metrics for DSV2 with output limits") - .exclude("UDF input_file_name()") - .exclude("Option recursiveFileLookup: disable partition inferring") + .includeCH("Return correct results when data columns overlap with partition columns") + .includeCH("Return correct results when data columns overlap with partition " + + "columns (nested data)") .exclude("SPARK-31116: Select nested schema with case insensitive mode") - .exclude("SPARK-35669: special char in CSV header with filter pushdown") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type .exclude("File source v2: support passing data filters to FileScan without partitionFilters") // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type .exclude("File source v2: support partition pruning") // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type .exclude("SPARK-41017: filter pushdown with nondeterministic predicates") - .excludeGlutenTest("Spark native readers should respect spark.sql.caseSensitive - parquet") - .excludeGlutenTest("SPARK-25237 compute correct input metrics in FileScanRDD") - .excludeGlutenTest("Option recursiveFileLookup: disable partition inferring") + .excludeCH("SPARK-23072 Write and read back unicode column names - csv") + .excludeCH("Enabling/disabling ignoreMissingFiles using csv") + .excludeCH("SPARK-30362: test input metrics for DSV2") + .excludeCH("SPARK-35669: special char in CSV header with filter pushdown") + .excludeCH("Gluten - Spark native readers should respect spark.sql.caseSensitive - parquet") + .excludeCH("Gluten - SPARK-25237 compute correct input metrics in FileScanRDD") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using orc") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using parquet") + enableSuite[GlutenFileDataSourceV2FallBackSuite] + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("Fallback Parquet V2 to V1") + enableSuite[GlutenFileFormatWriterSuite] + // TODO: fix "empty file should be skipped while write to file" + .excludeCH("empty file should be skipped while write to file") + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] enableSuite[GlutenFileScanSuite] enableSuite[GlutenFileSourceCharVarcharTestSuite] - .exclude("char type values should be padded or trimmed: partitioned columns") - .exclude("varchar type values length check and trim: partitioned columns") - .exclude("char/varchar type values length check: partitioned columns of other types") - .exclude("char type comparison: partitioned columns") + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in map key") + .includeCH("length check for input string values: nested in map value") + .includeCH("length check for input string values: nested in both map key and value") + .includeCH("length check for input string values: nested in array of struct") + .includeCH("length check for input string values: nested in array of array") + enableSuite[GlutenFileSourceCustomMetadataStructSuite] enableSuite[GlutenFileSourceSQLInsertTestSuite] - .exclude("SPARK-33474: Support typed literals as partition spec values") - .exclude( + .excludeCH("SPARK-33474: Support typed literals as partition spec values") + .excludeCH( "SPARK-34556: checking duplicate static partition columns should respect case sensitive conf") + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + .excludeCH("unpartitioned table, single partition") + .excludeCH("SPARK-32019: Add spark.sql.files.minPartitionNum config") + .excludeCH( + "SPARK-32352: Partially push down support data filter if it mixed in partition filters") + .excludeCH("SPARK-44021: Test spark.sql.files.maxPartitionNum works as expected") + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] enableSuite[GlutenGeneratorFunctionSuite] - .exclude("single explode_outer") - .exclude("single posexplode") - .exclude("single posexplode_outer") - .exclude("explode_outer and other columns") - .exclude("aliased explode_outer") - .exclude("explode_outer on map") - .exclude("explode_outer on map with aliases") - .exclude("inline_outer") - .exclude("SPARK-14986: Outer lateral view with empty generate expression") - .exclude("outer explode()") - .exclude("generator in aggregate expression") - .exclude("SPARK-37947: lateral view _outer()") - enableSuite[GlutenInjectRuntimeFilterSuite].exclude("Merge runtime bloom filters") + .exclude("SPARK-45171: Handle evaluated nondeterministic expression") + .excludeCH("single explode_outer") + .excludeCH("single posexplode_outer") + .excludeCH("explode_outer and other columns") + .excludeCH("aliased explode_outer") + .excludeCH("explode_outer on map") + .excludeCH("explode_outer on map with aliases") + .excludeCH("SPARK-40963: generator output has correct nullability") + .excludeCH("Gluten - SPARK-45171: Handle evaluated nondeterministic expression") + enableSuite[GlutenGroupBasedDeleteFromTableSuite] + enableSuite[GlutenGroupBasedMergeIntoTableSuite] + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenHashExpressionsSuite] + .excludeCH("sha2") + .excludeCH("SPARK-30633: xxHash with different type seeds") + enableSuite[GlutenHeaderCSVReadSchemaSuite] + .excludeCH("append column at the end") + .excludeCH("hide column at the end") + .excludeCH("change column type from byte to short/int/long") + .excludeCH("change column type from short to int/long") + .excludeCH("change column type from int to long") + .excludeCH("read byte, int, short, long together") + .excludeCH("change column type from float to double") + .excludeCH("read float and double together") + .excludeCH("change column type from float to decimal") + .excludeCH("change column type from double to decimal") + .excludeCH("read float, double, decimal together") + .excludeCH("read as string") + enableSuite[GlutenHigherOrderFunctionsSuite] + .excludeCH("ArraySort") + .excludeCH("ArrayAggregate") + .excludeCH("TransformKeys") + .excludeCH("TransformValues") + .excludeCH("SPARK-39419: ArraySort should throw an exception when the comparator returns null") + enableSuite[GlutenHiveSQLQueryCHSuite] + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .includeCH("Merge runtime bloom filters") + enableSuite[GlutenInnerJoinSuiteForceShjOff] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInnerJoinSuiteForceShjOn] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInsertSuite] + // the native write staing dir is differnt with vanilla Spark for coustom partition paths + .exclude("SPARK-35106: Throw exception when rename custom partition paths returns false") + .exclude("Stop task set if FileAlreadyExistsException was thrown") + // Rewrite: Additional support for file scan with default values has been added in Spark-3.4. + // It appends the default value in record if it is not present while scanning. + // Velox supports default values for new records but it does not backfill the + // existing records and provides null for the existing ones. + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + .excludeCH("Gluten - insert partition table") + .excludeCH("Gluten - remove v1writes sort and project") + .excludeCH("Gluten - remove v1writes sort") + .excludeCH("Gluten - do not remove non-v1writes sort and project") + .excludeCH( + "Gluten - SPARK-35106: Throw exception when rename custom partition paths returns false") + .excludeCH( + "Gluten - Do not fallback write files if output columns contain Spark internal metadata") + .excludeCH("Gluten - Add metadata white list to allow native write files") + .excludeCH("Gluten - INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + enableSuite[GlutenIntervalExpressionsSuite] enableSuite[GlutenIntervalFunctionsSuite] - enableSuite[GlutenJoinSuite].exclude( - "SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + .excludeCH( + "SPARK-45882: BroadcastHashJoinExec propagate partitioning should respect CoalescedHashPartitioning") enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .includeCH("$.store.book") + .includeCH("$") + .includeCH("$.store.book[0]") + .includeCH("$.store.book[*]") + .includeCH("$.store.book[*].category") + .includeCH("$.store.book[*].isbn") + .includeCH("$.store.book[*].reader") + .includeCH("$.store.basket[*]") + .includeCH("$.store.basket[*][0]") + .includeCH("$.store.basket[0][*]") + .includeCH("$.store.basket[*][*]") .exclude("$.store.basket[0][*].b") + // Exception class different. .exclude("from_json - invalid data") - .exclude("from_json - input=object, schema=array, output=array of single row") - .exclude("from_json - input=empty object, schema=array, output=array of single row with null") - .exclude("from_json - input=array of single object, schema=struct, output=single row") - .exclude("from_json - input=array, schema=struct, output=single row") - .exclude("from_json - input=empty array, schema=struct, output=single row with null") - .exclude("from_json - input=empty object, schema=struct, output=single row with null") - .exclude("SPARK-20549: from_json bad UTF-8") - .exclude("from_json with timestamp") - .exclude("to_json - struct") - .exclude("to_json - array") - .exclude("to_json - array with single empty row") - .exclude("to_json with timestamp") - .exclude("SPARK-21513: to_json support map[string, struct] to json") - .exclude("SPARK-21513: to_json support map[struct, struct] to json") - .exclude("parse date with locale") - .exclude("parse decimals using locale") - // NOT use gluten - .exclude("$..no_recursive") - .exclude("non foldable literal") - .exclude("some big value") - .exclude("from/to json - interval support") - .exclude("SPARK-24709: infer schema of json strings") - .exclude("infer schema of JSON strings by using options") - .exclude("inferring the decimal type using locale") - .exclude("json_object_keys") + .excludeCH("from_json - input=object, schema=array, output=array of single row") + .excludeCH("from_json - input=empty object, schema=array, output=array of single row with null") + .excludeCH("from_json - input=array of single object, schema=struct, output=single row") + .excludeCH("from_json - input=array, schema=struct, output=single row") + .excludeCH("from_json - input=empty array, schema=struct, output=single row with null") + .excludeCH("from_json - input=empty object, schema=struct, output=single row with null") + .excludeCH("SPARK-20549: from_json bad UTF-8") + .excludeCH("from_json with timestamp") + .excludeCH("to_json - struct") + .excludeCH("to_json - array") + .excludeCH("to_json - array with single empty row") + .excludeCH("to_json with timestamp") + .excludeCH("SPARK-21513: to_json support map[string, struct] to json") + .excludeCH("SPARK-21513: to_json support map[struct, struct] to json") + .excludeCH("parse date with locale") + .excludeCH("parse decimals using locale") enableSuite[GlutenJsonFunctionsSuite] - .exclude("from_json with option") - .exclude("from_json missing columns") - .exclude("from_json invalid json") - .exclude("from_json array support") - .exclude("to_json with option") - .exclude("roundtrip in to_json and from_json - array") - .exclude("SPARK-19637 Support to_json in SQL") - .exclude("SPARK-19967 Support from_json in SQL") - .exclude("pretty print - roundtrip from_json -> to_json") - .exclude("from_json invalid json - check modes") - .exclude("SPARK-36069: from_json invalid json schema - check field name and field value") - .exclude("corrupt record column in the middle") - .exclude("parse timestamps with locale") - .exclude("from_json - timestamp in micros") - .exclude("SPARK-33134: return partial results only for root JSON objects") - .exclude("SPARK-33907: bad json input with json pruning optimization: GetStructField") - .exclude("SPARK-33907: json pruning optimization with corrupt record field") - .exclude("SPARK-33907: bad json input with json pruning optimization: GetArrayStructFields") - enableSuite[GlutenMathFunctionsSuite].exclude("csc").exclude("sec") - enableSuite[GlutenMetadataCacheSuite].exclude( - "SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") - enableSuite[GlutenMiscFunctionsSuite] - enableSuite[GlutenNestedDataSourceV1Suite] - enableSuite[GlutenNestedDataSourceV2Suite] - enableSuite[GlutenProcessingTimeSuite] - enableSuite[GlutenProductAggSuite] - enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] - enableSuite[GlutenSQLQuerySuite] - .exclude("self join with alias in agg") - .exclude("SPARK-3176 Added Parser of SQL LAST()") - .exclude("SPARK-3173 Timestamp support in the parser") - .exclude("SPARK-11111 null-safe join should not use cartesian product") - .exclude("SPARK-3349 partitioning after limit") - .exclude("aggregation with codegen updates peak execution memory") - .exclude("SPARK-10215 Div of Decimal returns null") - .exclude("precision smaller than scale") - .exclude("external sorting updates peak execution memory") - .exclude("run sql directly on files") - .exclude("Struct Star Expansion") - .exclude("Common subexpression elimination") - .exclude( - "SPARK-27619: When spark.sql.legacy.allowHashOnMapType is true, hash can be used on Maptype") - .exclude("SPARK-24940: coalesce and repartition hint") - .exclude("SPARK-25144 'distinct' causes memory leak") - .exclude("SPARK-29239: Subquery should not cause NPE when eliminating subexpression") - .exclude("normalize special floating numbers in subquery") - .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") - .exclude("SPARK-33593: Vector reader got incorrect data with binary partition value") - .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") - .exclude("SPARK-27442: Spark support read/write parquet file with invalid char in field name") - .exclude("SPARK-37965: Spark support read/write orc file with invalid char in field name") - .exclude("SPARK-38548: try_sum should return null if overflow happens before merging") - .exclude("SPARK-38589: try_avg should return null if overflow happens before merging") - .exclude("SPARK-39548: CreateView will make queries go into inline CTE code path thustrigger a mis-clarified `window definition not found` issue") - .excludeGlutenTest("SPARK-33593: Vector reader got incorrect data with binary partition value") - .excludeGlutenTest( - "SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") - enableSuite[GlutenSQLQueryTestSuite] - enableSuite[GlutenScalaReflectionRelationSuite] - enableSuite[GlutenSerializationSuite] - enableSuite[GlutenStatisticsCollectionSuite] - .exclude("analyze empty table") - .exclude("analyze column command - result verification") - .exclude("column stats collection for null columns") - .exclude("store and retrieve column stats in different time zones") - .excludeGlutenTest("store and retrieve column stats in different time zones") - enableSuite[GlutenStringFunctionsSuite] - .exclude("string regex_replace / regex_extract") - .exclude("string overlay function") - .exclude("binary overlay function") - .exclude("string parse_url function") - .exclude("string / binary length function") - .exclude("SPARK-36751: add octet length api for scala") - .exclude("SPARK-36751: add bit length api for scala") - enableSuite[GlutenSubquerySuite] - .exclude("SPARK-15370: COUNT bug in subquery in subquery in subquery") - .exclude("SPARK-26893: Allow pushdown of partition pruning subquery filters to file source") - .exclude("SPARK-28441: COUNT bug in nested subquery with non-foldable expr") - .exclude("SPARK-28441: COUNT bug with non-foldable expression in Filter condition") - .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") - .exclude("SPARK-36656: Do not collapse projects with correlate scalar subqueries") - .exclude("Merge non-correlated scalar subqueries from different parent plans") - .exclude("Merge non-correlated scalar subqueries with conflicting names") - enableSuite[GlutenTypedImperativeAggregateSuite] - enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite].exclude("cases when literal is max") - enableSuite[GlutenXPathFunctionsSuite] - enableSuite[QueryTestSuite] - enableSuite[GlutenArithmeticExpressionSuite] - .exclude("- (UnaryMinus)") - .exclude("/ (Divide) basic") - .exclude("/ (Divide) for Long and Decimal type") - .exclude("% (Remainder)") - .exclude("SPARK-17617: % (Remainder) double % double on super big double") - .exclude("Abs") - .exclude("pmod") - .exclude("SPARK-28322: IntegralDivide supports decimal type") - .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") - .exclude("SPARK-34920: error class") - .exclude("SPARK-36920: Support year-month intervals by ABS") - .exclude("SPARK-36920: Support day-time intervals by ABS") - .exclude("SPARK-36921: Support YearMonthIntervalType by div") - .exclude("SPARK-36921: Support DayTimeIntervalType by div") - enableSuite[GlutenBitwiseExpressionsSuite] - enableSuite[GlutenCastSuite] - .exclude("null cast") - .exclude("cast string to date") - .exclude("cast string to timestamp") - .exclude("cast from boolean") - .exclude("data type casting") - .excludeGlutenTest("data type casting") - .exclude("cast between string and interval") - .exclude("SPARK-27671: cast from nested null type in struct") - .exclude("Process Infinity, -Infinity, NaN in case insensitive manner") - .exclude("SPARK-22825 Cast array to string") - .exclude("SPARK-33291: Cast array with null elements to string") - .exclude("SPARK-22973 Cast map to string") - .exclude("SPARK-22981 Cast struct to string") - .exclude("SPARK-33291: Cast struct with null elements to string") - .exclude("SPARK-34667: cast year-month interval to string") - .exclude("SPARK-34668: cast day-time interval to string") - .exclude("SPARK-35698: cast timestamp without time zone to string") - .exclude("SPARK-35711: cast timestamp without time zone to timestamp with local time zone") - .exclude("SPARK-35716: cast timestamp without time zone to date type") - .exclude("SPARK-35718: cast date type to timestamp without timezone") - .exclude("SPARK-35719: cast timestamp with local time zone to timestamp without timezone") - .exclude("SPARK-35720: cast string to timestamp without timezone") - .exclude("SPARK-35112: Cast string to day-time interval") - .exclude("SPARK-35111: Cast string to year-month interval") - .exclude("SPARK-35820: Support cast DayTimeIntervalType in different fields") - .exclude("SPARK-35819: Support cast YearMonthIntervalType in different fields") - .exclude("SPARK-35768: Take into account year-month interval fields in cast") - .exclude("SPARK-35735: Take into account day-time interval fields in cast") - .exclude("null cast #2") - .exclude("cast string to date #2") - .exclude("casting to fixed-precision decimals") - .exclude("SPARK-28470: Cast should honor nullOnOverflow property") - .exclude("cast string to boolean II") - .exclude("cast from array II") - .exclude("cast from map II") - .exclude("cast from struct II") - .exclude("cast from date") - .exclude("cast from timestamp II") - .exclude("cast a timestamp before the epoch 1970-01-01 00:00:00Z") - .exclude("SPARK-32828: cast from a derived user-defined type to a base type") - .exclude("SPARK-34727: cast from float II") - .exclude("SPARK-35720: cast invalid string input to timestamp without time zone") - .exclude("SPARK-36924: Cast DayTimeIntervalType to IntegralType") - .exclude("SPARK-36924: Cast IntegralType to DayTimeIntervalType") - .exclude("SPARK-36924: Cast YearMonthIntervalType to IntegralType") - .exclude("SPARK-36924: Cast IntegralType to YearMonthIntervalType") - enableSuite[GlutenCollectionExpressionsSuite] - .exclude("ArraysZip") // wait for https://github.com/ClickHouse/ClickHouse/pull/69576 - .exclude("Sequence of numbers") - .exclude("Shuffle") - .exclude("SPARK-33386: element_at ArrayIndexOutOfBoundsException") - .exclude("SPARK-33460: element_at NoSuchElementException") - .exclude("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") - .exclude( - "SPARK-36740: ArrayMin/ArrayMax/SortArray should handle NaN greater then non-NaN value") - .excludeGlutenTest("Shuffle") - enableSuite[GlutenComplexTypeSuite] - enableSuite[GlutenConditionalExpressionSuite] - .exclude("case when") - .exclude("if/case when - null flags of non-primitive types") - enableSuite[GlutenDateExpressionsSuite] - .exclude("DayOfYear") - .exclude("Year") - .exclude("Quarter") - .exclude("Month") - .exclude("Day / DayOfMonth") - .exclude("Seconds") - .exclude("DayOfWeek") - .exclude("WeekDay") - .exclude("WeekOfYear") - .exclude("DateFormat") - .exclude("Hour") - .exclude("Minute") - .exclude("date add interval") - .exclude("time_add") - .exclude("time_sub") - .exclude("add_months") - .exclude("SPARK-34721: add a year-month interval to a date") - .exclude("months_between") - .exclude("next_day") - .exclude("TruncDate") - .exclude("TruncTimestamp") - .exclude("unsupported fmt fields for trunc/date_trunc results null") - .exclude("from_unixtime") - .excludeGlutenTest("from_unixtime") - .exclude("unix_timestamp") - .exclude("to_unix_timestamp") - .exclude("to_utc_timestamp") - .exclude("from_utc_timestamp") - .exclude("creating values of DateType via make_date") - .exclude("creating values of Timestamp/TimestampNTZ via make_timestamp") - .exclude("ISO 8601 week-numbering year") - .exclude("extract the seconds part with fraction from timestamps") - .exclude("SPARK-34903: timestamps difference") - .exclude("SPARK-35916: timestamps without time zone difference") - .exclude("SPARK-34896: subtract dates") - .exclude("to_timestamp_ntz") - .exclude("to_timestamp exception mode") - .exclude("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") - .exclude("DATE_FROM_UNIX_DATE") - .exclude("UNIX_SECONDS") - .exclude("TIMESTAMP_SECONDS") // refer to https://github.com/ClickHouse/ClickHouse/issues/69280 - .exclude("TIMESTAMP_MICROS") // refer to https://github.com/apache/incubator-gluten/issues/7127 - .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") - .exclude("SPARK-34739,SPARK-35889: add a year-month interval to a timestamp") - .exclude("SPARK-34761,SPARK-35889: add a day-time interval to a timestamp") - .exclude("SPARK-37552: convert a timestamp_ntz to another time zone") - .exclude("SPARK-38195: add a quantity of interval units to a timestamp") - .exclude("SPARK-38284: difference between two timestamps in units") - .excludeGlutenTest("unix_timestamp") - .excludeGlutenTest("to_unix_timestamp") - .excludeGlutenTest("Hour") - enableSuite[GlutenDecimalExpressionSuite].exclude("MakeDecimal") - enableSuite[GlutenDecimalPrecisionSuite] - enableSuite[GlutenHashExpressionsSuite] - .exclude("sha2") - .exclude("murmur3/xxHash64/hive hash: struct") - .exclude("SPARK-30633: xxHash64 with long seed: struct") - .exclude("murmur3/xxHash64/hive hash: struct,arrayOfString:array,arrayOfArrayOfString:array>,arrayOfArrayOfInt:array>,arrayOfStruct:array>,arrayOfUDT:array>") - .exclude("SPARK-30633: xxHash64 with long seed: struct,arrayOfString:array,arrayOfArrayOfString:array>,arrayOfArrayOfInt:array>,arrayOfStruct:array>,arrayOfUDT:array>") - .exclude("murmur3/xxHash64/hive hash: struct,structOfStructOfString:struct>,structOfArray:struct>,structOfUDT:struct>") - .exclude("SPARK-30633: xxHash64 with long seed: struct,structOfStructOfString:struct>,structOfArray:struct>,structOfUDT:struct>") - .exclude("SPARK-30633: xxHash with different type seeds") - .exclude("SPARK-35113: HashExpression support DayTimeIntervalType/YearMonthIntervalType") - .exclude("SPARK-35207: Compute hash consistent between -0.0 and 0.0") - enableSuite[GlutenIntervalExpressionsSuite] - .exclude("years") - .exclude("months") - .exclude("days") - .exclude("hours") - .exclude("minutes") - .exclude("seconds") - .exclude("multiply") - .exclude("divide") - .exclude("make interval") - .exclude("ANSI mode: make interval") - .exclude("SPARK-35130: make day time interval") - .exclude("SPARK-34824: multiply year-month interval by numeric") - .exclude("SPARK-34850: multiply day-time interval by numeric") - .exclude("SPARK-34868: divide year-month interval by numeric") - .exclude("SPARK-34875: divide day-time interval by numeric") - .exclude("ANSI: extract years and months") - .exclude("ANSI: extract days, hours, minutes and seconds") - .exclude("SPARK-35129: make_ym_interval") - .exclude("SPARK-35728: Check multiply/divide of day-time intervals of any fields by numeric") - .exclude("SPARK-35778: Check multiply/divide of year-month intervals of any fields by numeric") + // * in get_json_object expression not supported in velox + .exclude("SPARK-42782: Hive compatibility check for get_json_object") + // Velox does not support single quotes in get_json_object function. + .includeCH("function get_json_object - support single quotes") + .excludeCH("from_json with option (allowComments)") + .excludeCH("from_json with option (allowUnquotedFieldNames)") + .excludeCH("from_json with option (allowSingleQuotes)") + .excludeCH("from_json with option (allowNumericLeadingZeros)") + .excludeCH("from_json with option (allowBackslashEscapingAnyCharacter)") + .excludeCH("from_json with option (dateFormat)") + .excludeCH("from_json with option (allowUnquotedControlChars)") + .excludeCH("from_json with option (allowNonNumericNumbers)") + .excludeCH("from_json missing columns") + .excludeCH("from_json invalid json") + .excludeCH("from_json array support") + .excludeCH("to_json with option (timestampFormat)") + .excludeCH("to_json with option (dateFormat)") + .excludeCH("SPARK-19637 Support to_json in SQL") + .excludeCH("pretty print - roundtrip from_json -> to_json") + .excludeCH("from_json invalid json - check modes") + .excludeCH("SPARK-36069: from_json invalid json schema - check field name and field value") + .excludeCH("corrupt record column in the middle") + .excludeCH("parse timestamps with locale") + .excludeCH("SPARK-33134: return partial results only for root JSON objects") + .excludeCH("SPARK-40646: return partial results for JSON arrays with objects") + .excludeCH("SPARK-40646: return partial results for JSON maps") + .excludeCH("SPARK-40646: return partial results for objects with values as JSON arrays") + .excludeCH("SPARK-48863: parse object as an array with partial results enabled") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetStructField") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetArrayStructFields") + .excludeCH("SPARK-33907: json pruning optimization with corrupt record field") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonV2Suite] + // exception test + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenKeyGroupedPartitioningSuite] + // NEW SUITE: disable as they check vanilla spark plan + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + // disable due to check for SMJ node + .excludeByPrefix("SPARK-41413: partitioned join:") + .excludeByPrefix("SPARK-42038: partially clustered:") + .exclude("SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH("Gluten - partitioned join: only one side reports partitioning") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from one side are subset of those from the other side") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from both sides overlaps") + .excludeCH( + "Gluten - SPARK-41413: partitioned join: non-overlapping partition values from both sides") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and both sides partially clustered") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on left-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on right-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: left outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: right outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: full outer join is not applicable") + .excludeCH("Gluten - SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH( + "Gluten - partitioned join: join with two partition keys and different # of partition keys") + enableSuite[GlutenLateralColumnAliasSuite] + .excludeCH("Lateral alias conflicts with table column - Project") + .excludeCH("Lateral alias conflicts with table column - Aggregate") + .excludeCH("Lateral alias of a complex type") + .excludeCH("Lateral alias reference works with having and order by") + .excludeCH("Lateral alias basics - Window on Project") + .excludeCH("Lateral alias basics - Window on Aggregate") enableSuite[GlutenLiteralExpressionSuite] - .exclude("null") .exclude("default") - .exclude("decimal") - .exclude("array") - .exclude("seq") - .exclude("map") - .exclude("struct") - .exclude("SPARK-35664: construct literals from java.time.LocalDateTime") - .exclude("SPARK-34605: construct literals from java.time.Duration") - .exclude("SPARK-34605: construct literals from arrays of java.time.Duration") - .exclude("SPARK-34615: construct literals from java.time.Period") - .exclude("SPARK-34615: construct literals from arrays of java.time.Period") - .exclude("SPARK-35871: Literal.create(value, dataType) should support fields") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenLocalBroadcastExchangeSuite] + .excludeCH("SPARK-39983 - Broadcasted relation is not cached on the driver") + enableSuite[GlutenLocalScanSuite] enableSuite[GlutenMathExpressionsSuite] - .exclude("csc") - .exclude("sec") - .exclude("cot") - .exclude("tanh") - .exclude("ceil") - .exclude("floor") - .exclude("factorial") - .exclude("rint") - .exclude("expm1") - .exclude("log") - .exclude("log10") - .exclude("bin") - .exclude("log2") - .exclude("unhex") - .exclude("atan2") - .exclude("binary log") + // Spark round UT for round(3.1415,3) is not correct. .exclude("round/bround/floor/ceil") - .excludeGlutenTest("round/bround/floor/ceil") - .exclude("SPARK-36922: Support ANSI intervals for SIGN/SIGNUM") - .exclude("SPARK-35926: Support YearMonthIntervalType in width-bucket function") - .exclude("SPARK-35925: Support DayTimeIntervalType in width-bucket function") - .exclude("SPARK-37388: width_bucket") - .exclude("shift left") - .exclude("shift right") - .exclude("shift right unsigned") + .excludeCH("tanh") + .excludeCH("unhex") + .excludeCH("atan2") + .excludeCH("SPARK-42045: integer overflow in round/bround") + .excludeCH("Gluten - round/bround/floor/ceil") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("read byte, int, short, long together") + .includeCH("change column type from float to double") + .includeCH("read float and double together") + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMetadataColumnSuite] + .excludeCH("SPARK-34923: propagate metadata columns through Sort") + .excludeCH("SPARK-34923: propagate metadata columns through RepartitionBy") + .excludeCH("SPARK-40149: select outer join metadata columns with DataFrame API") + .excludeCH("SPARK-42683: Project a metadata column by its logical name - column not found") enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] enableSuite[GlutenNondeterministicSuite] .exclude("MonotonicallyIncreasingID") - .exclude("SparkPartitionID") - .exclude("InputFileName") - enableSuite[GlutenNullExpressionsSuite] - .exclude("nanvl") - .exclude("AtLeastNNonNulls") - .exclude("AtLeastNNonNulls should not throw 64KiB exception") - enableSuite[GlutenPredicateSuite] - .exclude("3VL Not") - .exclude("3VL AND") - .exclude("3VL OR") - .exclude("3VL =") - .exclude("basic IN/INSET predicate test") - .exclude("IN with different types") - .exclude("IN/INSET: binary") - .exclude("IN/INSET: struct") - .exclude("IN/INSET: array") - .exclude("BinaryComparison: lessThan") - .exclude("BinaryComparison: LessThanOrEqual") - .exclude("BinaryComparison: GreaterThan") - .exclude("BinaryComparison: GreaterThanOrEqual") - .exclude("BinaryComparison: EqualTo") - .exclude("BinaryComparison: EqualNullSafe") - .exclude("BinaryComparison: null test") - .exclude("EqualTo on complex type") - .exclude("isunknown and isnotunknown") - .exclude("SPARK-32764: compare special double/float values") - .exclude("SPARK-32110: compare special double/float values in array") - .exclude("SPARK-32110: compare special double/float values in struct") - enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") - enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE Pattern") - .exclude("LIKE Pattern ESCAPE '/'") - .exclude("LIKE Pattern ESCAPE '#'") - .exclude("LIKE Pattern ESCAPE '\"'") - .exclude("RLIKE Regular Expression") - .exclude("RegexReplace") - .exclude("RegexExtract") - .exclude("RegexExtractAll") - enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") - enableSuite[GlutenStringExpressionsSuite] - .exclude("StringComparison") - .exclude("Substring") - .exclude("string substring_index function") - .exclude("SPARK-40213: ascii for Latin-1 Supplement characters") - .exclude("ascii for string") - .exclude("base64/unbase64 for string") - .exclude("encode/decode for string") - .exclude("Levenshtein distance") - .exclude("overlay for string") - .exclude("overlay for byte array") - .exclude("translate") - .exclude("LOCATE") - .exclude("REPEAT") - .exclude("ParseUrl") - .exclude("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") - .exclude("FORMAT") // refer https://github.com/apache/incubator-gluten/issues/6765 - .exclude( - "soundex unit test" - ) // CH and spark returns different results when input non-ASCII characters - enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] - enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] - enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] - enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] - enableSuite[GlutenDataSourceV2Suite] - .exclude("partitioning reporting") - .exclude("SPARK-33267: push down with condition 'in (..., null)' should not throw NPE") - enableSuite[GlutenDeleteFromTableSuite] - enableSuite[GlutenFileDataSourceV2FallBackSuite] - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude("Fallback Parquet V2 to V1") - enableSuite[GlutenKeyGroupedPartitioningSuite] - .exclude("partitioned join: number of buckets mismatch should trigger shuffle") - .exclude("partitioned join: only one side reports partitioning") - .exclude("partitioned join: join with two partition keys and different # of partition keys") - enableSuite[GlutenLocalScanSuite] - enableSuite[GlutenSupportsCatalogOptionsSuite] - enableSuite[GlutenTableCapabilityCheckSuite] - enableSuite[GlutenWriteDistributionAndOrderingSuite] - enableSuite[GlutenQueryCompilationErrorsSuite] - .exclude("CANNOT_USE_MIXTURE: Using aggregate function with grouped aggregate pandas UDF") - .exclude("UNSUPPORTED_FEATURE: Using pandas UDF aggregate expression with pivot") - enableSuite[GlutenQueryExecutionErrorsSuite] - .exclude( - "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") - .exclude("UNSUPPORTED_OPERATION - SPARK-38504: can't read TimestampNTZ as TimestampLTZ") - enableSuite[GlutenQueryParsingErrorsSuite] - enableSuite[FallbackStrategiesSuite] - enableSuite[GlutenBroadcastExchangeSuite] - enableSuite[GlutenCoalesceShufflePartitionsSuite] - .exclude( - "determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") - .exclude("determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") - .exclude("determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") - .exclude("determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") - .exclude( - "determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") - .exclude("determining the number of reducers: aggregate operator") - .exclude("determining the number of reducers: join operator") - .exclude("determining the number of reducers: complex query 1") - .exclude("determining the number of reducers: complex query 2") - .exclude("determining the number of reducers: plan already partitioned") - .exclude("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") - .exclude("Do not reduce the number of shuffle partition for repartition") - .exclude("Union two datasets with different pre-shuffle partition number") - .exclude("SPARK-34790: enable IO encryption in AQE partition coalescing") - .excludeGlutenTest( - "SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") - .excludeGlutenTest("SPARK-34790: enable IO encryption in AQE partition coalescing") - .excludeGlutenTest( - "determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") - .excludeGlutenTest( - "determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") - .excludeGlutenTest( - "determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") - .excludeGlutenTest( - "determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") - .excludeGlutenTest( - "determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") - .excludeGlutenTest("determining the number of reducers: aggregate operator") - .excludeGlutenTest("determining the number of reducers: join operator") - .excludeGlutenTest("determining the number of reducers: complex query 1") - .excludeGlutenTest("determining the number of reducers: complex query 2") - .excludeGlutenTest("determining the number of reducers: plan already partitioned") - enableSuite[GlutenExchangeSuite] - .exclude("shuffling UnsafeRows in exchange") - .exclude("SPARK-23207: Make repartition() generate consistent output") - .exclude("Exchange reuse across the whole plan") - enableSuite[GlutenReplaceHashWithSortAggSuite] - .exclude("replace partial hash aggregate with sort aggregate") - .exclude("replace partial and final hash aggregate together with sort aggregate") - .exclude("do not replace hash aggregate if child does not have sort order") - .exclude("do not replace hash aggregate if there is no group-by column") - .excludeGlutenTest("replace partial hash aggregate with sort aggregate") - enableSuite[GlutenReuseExchangeAndSubquerySuite] - enableSuite[GlutenSQLAggregateFunctionSuite] - enableSuite[GlutenSQLWindowFunctionSuite] - .exclude("window function: partition and order expressions") - .exclude("window function: expressions in arguments of a window functions") - .exclude( - "window function: multiple window expressions specified by range in a single expression") - .exclude("SPARK-7595: Window will cause resolve failed with self join") - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") - .exclude("lead/lag should respect null values") - .exclude("test with low buffer spill threshold") - enableSuite[GlutenSameResultSuite] - enableSuite[GlutenSortSuite] - .exclude("basic sorting using ExternalSort") - .exclude("sort followed by limit") - .exclude("sorting does not crash for large inputs") - .exclude("sorting updates peak execution memory") - .exclude("SPARK-33260: sort order is a Stream") - .exclude("sorting on StringType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on StringType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on StringType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on StringType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on StringType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on StringType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on LongType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on LongType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on LongType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on LongType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on LongType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on LongType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on LongType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on LongType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude( - "sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DateType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DateType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DateType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on DateType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DateType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DateType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DateType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on DateType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude( - "sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude( - "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude( - "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude( - "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on ByteType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on ByteType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on ByteType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on ByteType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on ByteType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on ByteType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on ByteType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on ByteType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on FloatType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on FloatType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on FloatType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on FloatType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on FloatType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on FloatType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on FloatType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on FloatType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on ShortType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on ShortType with nullable=true, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on ShortType with nullable=true, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on ShortType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") - .exclude("sorting on ShortType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") - .exclude("sorting on ShortType with nullable=false, sortOrder=List('a ASC NULLS LAST)") - .exclude("sorting on ShortType with nullable=false, sortOrder=List('a DESC NULLS LAST)") - .exclude("sorting on ShortType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") - .excludeByPrefix("sorting on YearMonthIntervalType(0,1) with") - enableSuite[GlutenTakeOrderedAndProjectSuite] - .exclude("TakeOrderedAndProject.doExecute without project") - .exclude("TakeOrderedAndProject.doExecute with project") - enableSuite[ClickHouseAdaptiveQueryExecSuite] - .exclude("Change merge join to broadcast join") - .exclude("Reuse the parallelism of coalesced shuffle in local shuffle read") - .exclude("Reuse the default parallelism in local shuffle read") - .exclude("Empty stage coalesced to 1-partition RDD") - .exclude("Scalar subquery") - .exclude("Scalar subquery in later stages") - .exclude("multiple joins") - .exclude("multiple joins with aggregate") - .exclude("multiple joins with aggregate 2") - .exclude("Exchange reuse") - .exclude("Exchange reuse with subqueries") - .exclude("Exchange reuse across subqueries") - .exclude("Subquery reuse") - .exclude("Broadcast exchange reuse across subqueries") - .exclude("Change merge join to broadcast join without local shuffle read") - .exclude( - "Avoid changing merge join to broadcast join if too many empty partitions on build plan") - .exclude( - "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") - .exclude("SPARK-37753: Allow changing outer join to broadcast join even if too many empty partitions on broadcast side") - .exclude("SPARK-29544: adaptive skew join with different join types") - .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") - .exclude("SPARK-32717: AQEOptimizer should respect excludedRules configuration") - .exclude("metrics of the shuffle read") - .exclude("SPARK-31220, SPARK-32056: repartition by expression with AQE") - .exclude("SPARK-31220, SPARK-32056: repartition by range with AQE") - .exclude("SPARK-31220, SPARK-32056: repartition using sql and hint with AQE") - .exclude("SPARK-32753: Only copy tags to node with no tags") - .exclude("Logging plan changes for AQE") - .exclude("SPARK-33551: Do not use AQE shuffle read for repartition") - .exclude("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") - .exclude("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") - .exclude("SPARK-34980: Support coalesce partition through union") - .exclude("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") - .exclude("SPARK-35264: Support AQE side broadcastJoin threshold") - .exclude("SPARK-35264: Support AQE side shuffled hash join formula") - .exclude("SPARK-35650: Coalesce number of partitions by AEQ") - .exclude("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") - .exclude("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") - .exclude("SPARK-35888: join with a 0-partition table") - .exclude("SPARK-33832: Support optimize skew join even if introduce extra shuffle") - .exclude("SPARK-35968: AQE coalescing should not produce too small partitions by default") - .exclude("SPARK-35794: Allow custom plugin for cost evaluator") - .exclude("SPARK-36020: Check logical link in remove redundant projects") - .exclude("SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") - .exclude("SPARK-37063: OptimizeSkewInRebalancePartitions support optimize non-root node") - .exclude("SPARK-37357: Add small partition factor for rebalance partitions") - .exclude("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") - .exclude("SPARK-37328: skew join with 3 tables") - .exclude("SPARK-39915: Dataset.repartition(N) may not create N partitions") - .excludeGlutenTest("Change broadcast join to merge join") - .excludeGlutenTest("Empty stage coalesced to 1-partition RDD") - .excludeGlutenTest( - "Avoid changing merge join to broadcast join if too many empty partitions on build plan") - .excludeGlutenTest("SPARK-30524: Do not optimize skew join if introduce additional shuffle") - .excludeGlutenTest("SPARK-33551: Do not use AQE shuffle read for repartition") - .excludeGlutenTest("SPARK-35264: Support AQE side broadcastJoin threshold") - .excludeGlutenTest("SPARK-35264: Support AQE side shuffled hash join formula") - .excludeGlutenTest("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") - .excludeGlutenTest( - "SPARK-35968: AQE coalescing should not produce too small partitions by default") - .excludeGlutenTest( - "SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") - enableSuite[GlutenBucketingUtilsSuite] - enableSuite[GlutenCSVReadSchemaSuite] - enableSuite[GlutenDataSourceStrategySuite] - enableSuite[GlutenDataSourceSuite] - enableSuite[GlutenFileFormatWriterSuite].excludeByPrefix( - "empty file should be skipped while write to file") - enableSuite[GlutenFileIndexSuite] - enableSuite[GlutenFileMetadataStructSuite] - .exclude("metadata struct (json): file metadata in streaming") - .exclude("metadata struct (parquet): file metadata in streaming") - enableSuite[GlutenFileSourceStrategySuite] - .exclude("unpartitioned table, single partition") - .exclude("partitioned table - after scan filters") - .exclude("SPARK-32019: Add spark.sql.files.minPartitionNum config") - .exclude( - "SPARK-32352: Partially push down support data filter if it mixed in partition filters") - enableSuite[GlutenHadoopFileLinesReaderSuite] - enableSuite[GlutenHeaderCSVReadSchemaSuite] - .exclude("append column at the end") - .exclude("hide column at the end") - .exclude("change column type from byte to short/int/long") - .exclude("change column type from short to int/long") - .exclude("change column type from int to long") - .exclude("read byte, int, short, long together") - .exclude("change column type from float to double") - .exclude("read float and double together") - .exclude("change column type from float to decimal") - .exclude("change column type from double to decimal") - .exclude("read float, double, decimal together") - .exclude("read as string") - enableSuite[GlutenJsonReadSchemaSuite] - enableSuite[GlutenMergedOrcReadSchemaSuite] - enableSuite[GlutenMergedParquetReadSchemaSuite] - enableSuite[GlutenOrcCodecSuite] - enableSuite[GlutenOrcReadSchemaSuite] - enableSuite[GlutenOrcV1AggregatePushDownSuite].exclude( - "aggregate push down - different data types") - enableSuite[GlutenOrcV2AggregatePushDownSuite].exclude( - "aggregate push down - different data types") - enableSuite[GlutenParquetCodecSuite] - enableSuite[GlutenParquetReadSchemaSuite] - enableSuite[GlutenParquetV1AggregatePushDownSuite] - enableSuite[GlutenParquetV2AggregatePushDownSuite] - enableSuite[GlutenPathFilterStrategySuite] - enableSuite[GlutenPathFilterSuite] - enableSuite[GlutenPruneFileSourcePartitionsSuite] - enableSuite[GlutenVectorizedOrcReadSchemaSuite] - enableSuite[GlutenVectorizedParquetReadSchemaSuite] - enableSuite[GlutenBinaryFileFormatSuite] - .exclude("column pruning - non-readable file") - enableSuite[GlutenValidateRequirementsSuite] - enableSuite[GlutenJsonLegacyTimeParserSuite] - .exclude("Complex field and type inferring") - .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") - .exclude("SPARK-4228 DataFrame to JSON") - .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") - .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") - .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") - .exclude("SPARK-36830: Support reading and writing ANSI intervals") - enableSuite[GlutenJsonSuite] - .exclude("Complex field and type inferring") - .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") - .exclude("SPARK-4228 DataFrame to JSON") - .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") - .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") - .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") - .exclude("SPARK-36830: Support reading and writing ANSI intervals") - enableSuite[GlutenJsonV1Suite] - .exclude("Complex field and type inferring") - .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") - .exclude("SPARK-4228 DataFrame to JSON") - .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") - .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") - .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") - .exclude("SPARK-36830: Support reading and writing ANSI intervals") - enableSuite[GlutenJsonV2Suite] - .exclude("Complex field and type inferring") - .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") - .exclude("SPARK-4228 DataFrame to JSON") - .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") - .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") - .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") - .exclude("SPARK-36830: Support reading and writing ANSI intervals") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenOrcCodecSuite] enableSuite[GlutenOrcColumnarBatchReaderSuite] - enableSuite[GlutenOrcFilterSuite].exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") enableSuite[GlutenOrcPartitionDiscoverySuite] - enableSuite[GlutenOrcQuerySuite] - .exclude("Enabling/disabling ignoreCorruptFiles") - .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") - .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") - .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude( - "SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + enableSuite[GlutenOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("hide column in the middle") + .includeCH("change column position") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("read as string") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + .exclude("change column type from float to decimal") + .exclude("change column type from double to decimal") + .exclude("read float, double, decimal together") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") enableSuite[GlutenOrcSourceSuite] - .exclude("SPARK-24322 Fix incorrect workaround for bug in java.sql.Timestamp") + // Rewrite to disable Spark's columnar reader. .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") - .exclude("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") - .exclude("SPARK-37812: Reuse result row when deserializing a struct") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude("SPARK-34862: Support ORC vectorized reader for nested column") - .excludeByPrefix( - "SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false,") + .includeCH("SPARK-34862: Support ORC vectorized reader for nested column") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("create temporary orc table") + .includeCH("create temporary orc table as") + .includeCH("appending insert") + .includeCH("overwrite insert") + .includeCH("SPARK-34897: Support reconcile schemas based on index after nested column pruning") .excludeGlutenTest("SPARK-31238: compatibility with Spark 2.4 in reading dates") .excludeGlutenTest("SPARK-31238, SPARK-31423: rebasing dates in write") - .excludeGlutenTest("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") - .excludeGlutenTest("SPARK-31284, SPARK-31423: rebasing timestamps in write") .excludeGlutenTest("SPARK-34862: Support ORC vectorized reader for nested column") - .excludeGlutenTest( - "SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") - enableSuite[GlutenOrcV1FilterSuite].exclude("SPARK-32622: case sensitivity in predicate pushdown") + // exclude as struct not supported + .includeCH("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .includeCH("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("Gluten - SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .excludeCH("Gluten - SPARK-31284, SPARK-31423: rebasing timestamps in write") + .excludeCH("Gluten - SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1AggregatePushDownSuite] + .includeCH("nested column: Count(nested sub-field) not push down") enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + .includeCH("read partitioned table - partition key included in orc file") + .includeCH("read partitioned table - with nulls and partition keys are included in Orc file") enableSuite[GlutenOrcV1QuerySuite] - .exclude("Enabling/disabling ignoreCorruptFiles") - .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") - .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude( - "SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") enableSuite[GlutenOrcV1SchemaPruningSuite] - .exclude( - "Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + // Vectorized reading. + .includeCH("Spark vectorized reader - without partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - with partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") .exclude( "Case-insensitive parser - mixed-case schema - select with different-case column names") - .exclude( - "Case-insensitive parser - mixed-case schema - filter with different-case column names") .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") .exclude("SPARK-36352: Spark should check result plan's output schema name") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOrcV2AggregatePushDownSuite] + .includeCH("nested column: Max(top level column) not push down") + .includeCH("nested column: Count(nested sub-field) not push down") enableSuite[GlutenOrcV2QuerySuite] + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH( + "SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") .exclude("Enabling/disabling ignoreCorruptFiles") - .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") - .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") - .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude( - "SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") enableSuite[GlutenOrcV2SchemaPruningSuite] - .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") .exclude( "Case-insensitive parser - mixed-case schema - select with different-case column names") - .exclude( - "Case-insensitive parser - mixed-case schema - filter with different-case column names") .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") .exclude("SPARK-36352: Spark should check result plan's output schema name") - .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOuterJoinSuiteForceShjOff] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-32717: AQEOptimizer should respect excludedRules configuration") + enableSuite[GlutenOuterJoinSuiteForceShjOn] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + enableSuite[GlutenParametersSuite] + enableSuite[GlutenParquetCodecSuite] + // codec not supported in native + .includeCH("write and read - file source parquet - codec: lz4_raw") + .includeCH("write and read - file source parquet - codec: lz4raw") enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. .exclude("test reading unaligned pages - test all types") - .exclude("test reading unaligned pages - test all types (dict encode)") + .excludeCH("test reading unaligned pages - test all types (dict encode)") enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] enableSuite[GlutenParquetDeltaEncodingInteger] enableSuite[GlutenParquetDeltaEncodingLong] enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] - enableSuite[GlutenParquetEncodingSuite].exclude("All Types Dictionary").exclude("All Types Null") + enableSuite[GlutenParquetEncodingSuite] + // Velox does not support rle encoding, but it can pass when native writer enabled. + .includeCH("parquet v2 pages - rle encoding for boolean value columns") + .excludeCH("All Types Dictionary") + .excludeCH("All Types Null") enableSuite[GlutenParquetFieldIdIOSuite] enableSuite[GlutenParquetFileFormatV1Suite] - .exclude( + .excludeCH( "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") enableSuite[GlutenParquetFileFormatV2Suite] - .exclude( + .excludeCH( "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] enableSuite[GlutenParquetIOSuite] - .exclude("Standard mode - nested map with struct as key type") - .exclude("Legacy mode - nested map with struct as key type") - .exclude("vectorized reader: missing all struct fields") + // Velox doesn't write file metadata into parquet file. + .includeCH("Write Spark version into Parquet metadata") + // Exception. .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") - .exclude("SPARK-36726: test incorrect Parquet row group file offset") - enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") - enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") + // Velox parquet reader not allow offset zero. + .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetProtobufCompatibilitySuite] + .excludeCH("struct with unannotated array") + enableSuite[GlutenParquetReadSchemaSuite] enableSuite[GlutenParquetRebaseDatetimeV1Suite] - .exclude( - "SPARK-31159, SPARK-37705: compatibility with Spark 2.4/3.2 in reading dates/timestamps") - .exclude("SPARK-31159, SPARK-37705: rebasing timestamps in write") - .exclude("SPARK-31159: rebasing dates in write") - .exclude("SPARK-35427: datetime rebasing in the EXCEPTION mode") - .excludeGlutenTest("SPARK-31159: rebasing dates in write") + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + .excludeCH("Gluten - SPARK-31159: rebasing dates in write") enableSuite[GlutenParquetRebaseDatetimeV2Suite] - .exclude( - "SPARK-31159, SPARK-37705: compatibility with Spark 2.4/3.2 in reading dates/timestamps") - .exclude("SPARK-31159, SPARK-37705: rebasing timestamps in write") - .exclude("SPARK-31159: rebasing dates in write") - .exclude("SPARK-35427: datetime rebasing in the EXCEPTION mode") + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") enableSuite[GlutenParquetSchemaInferenceSuite] enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted .exclude("schema mismatch failure error message for parquet reader") .exclude("schema mismatch failure error message for parquet vectorized reader") + // [PATH_NOT_FOUND] Path does not exist: + // file:/opt/spark331/sql/core/src/test/resources/test-data/timestamp-nanos.parquet + // May require for newer spark.test.home + .excludeByPrefix("SPARK-40819") enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. .exclude("Read Parquet file generated by parquet-thrift") - .exclude("SPARK-10136 list of primitive list") + .excludeCH("SPARK-10136 list of primitive list") + enableSuite[GlutenParquetV1AggregatePushDownSuite] enableSuite[GlutenParquetV1FilterSuite] - .exclude("filter pushdown - date") + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") .exclude("Filters should be pushed down for Parquet readers at row group level") - .exclude("filter pushdown - StringStartsWith") .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + // avoid Velox compile error + enableSuite( + "org.apache.gluten.execution.parquet.GlutenParquetV1FilterSuite2" + ) + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") - .exclude("SPARK-38825: in and notIn filters") - .exclude("SPARK-36866: filter pushdown - year-month interval") - .exclude("filter pushdown - StringContains") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - .excludeGlutenTest("SPARK-25207: exception when duplicate fields in case-insensitive mode") + .excludeCH("filter pushdown - StringContains") enableSuite[GlutenParquetV1PartitionDiscoverySuite] - .exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping") - .exclude("Various partition value types") - .exclude("Various inferred partition value types") - .exclude( - "SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") - .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") enableSuite[GlutenParquetV1QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .includeCH("SPARK-39833: pushed filters with project without filter columns") + .includeCH("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") - .exclude("SPARK-34212 Parquet should read decimals correctly") enableSuite[GlutenParquetV1SchemaPruningSuite] - .exclude( - "Spark vectorized reader - without partition data column - select only top-level fields") - .exclude("Spark vectorized reader - with partition data column - select only top-level fields") - .exclude("Non-vectorized reader - without partition data column - select only top-level fields") - .exclude("Non-vectorized reader - with partition data column - select only top-level fields") - .exclude("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") - .exclude("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") - .exclude("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") - .exclude("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") - .exclude( - "Spark vectorized reader - without partition data column - select only input_file_name()") - .exclude("Spark vectorized reader - with partition data column - select only input_file_name()") - .exclude( - "Non-vectorized reader - without partition data column - select only input_file_name()") - .exclude("Non-vectorized reader - with partition data column - select only input_file_name()") - .exclude("Spark vectorized reader - without partition data column - select only expressions without references") - .exclude("Spark vectorized reader - with partition data column - select only expressions without references") - .exclude("Non-vectorized reader - without partition data column - select only expressions without references") - .exclude("Non-vectorized reader - with partition data column - select only expressions without references") - .exclude( - "Spark vectorized reader - without partition data column - select a single complex field") - .exclude("Spark vectorized reader - with partition data column - select a single complex field") - .exclude( - "Non-vectorized reader - without partition data column - select a single complex field") - .exclude("Non-vectorized reader - with partition data column - select a single complex field") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") - .exclude("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") - .exclude("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") - .exclude("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") - .exclude("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") - .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and the partition column") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and the partition column") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and the partition column") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and the partition column") - .exclude("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") - .exclude("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") - .exclude("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") - .exclude("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") - .exclude( - "Spark vectorized reader - without partition data column - no unnecessary schema pruning") - .exclude("Spark vectorized reader - with partition data column - no unnecessary schema pruning") - .exclude( - "Non-vectorized reader - without partition data column - no unnecessary schema pruning") - .exclude("Non-vectorized reader - with partition data column - no unnecessary schema pruning") - .exclude("Spark vectorized reader - without partition data column - empty schema intersection") - .exclude("Spark vectorized reader - with partition data column - empty schema intersection") - .exclude("Non-vectorized reader - without partition data column - empty schema intersection") - .exclude("Non-vectorized reader - with partition data column - empty schema intersection") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - without partition data column - select nullable complex field and having is not null predicate") - .exclude("Spark vectorized reader - with partition data column - select nullable complex field and having is not null predicate") - .exclude("Non-vectorized reader - without partition data column - select nullable complex field and having is not null predicate") - .exclude("Non-vectorized reader - with partition data column - select nullable complex field and having is not null predicate") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") - .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and in clause") - .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and in clause") - .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and in clause") - .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and in clause") - .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") - .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") - .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") - .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") - .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") - .exclude("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") - .exclude("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") - .exclude("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") - .exclude("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") - .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") - .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") - .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") - .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after join") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after join") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after join") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after join") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after outer join") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after outer join") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") - .exclude("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") - .exclude("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") - .exclude("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") - .exclude("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") - .exclude("Spark vectorized reader - without partition data column - select nested field in window function") - .exclude("Spark vectorized reader - with partition data column - select nested field in window function") - .exclude("Non-vectorized reader - without partition data column - select nested field in window function") - .exclude( - "Non-vectorized reader - with partition data column - select nested field in window function") - .exclude("Spark vectorized reader - without partition data column - select nested field in window function and then order by") - .exclude("Spark vectorized reader - with partition data column - select nested field in window function and then order by") - .exclude("Non-vectorized reader - without partition data column - select nested field in window function and then order by") - .exclude("Non-vectorized reader - with partition data column - select nested field in window function and then order by") - .exclude( - "Spark vectorized reader - without partition data column - select nested field in Sort") - .exclude("Spark vectorized reader - with partition data column - select nested field in Sort") - .exclude("Non-vectorized reader - without partition data column - select nested field in Sort") - .exclude("Non-vectorized reader - with partition data column - select nested field in Sort") - .exclude( - "Spark vectorized reader - without partition data column - select nested field in Expand") - .exclude("Spark vectorized reader - with partition data column - select nested field in Expand") - .exclude( - "Non-vectorized reader - without partition data column - select nested field in Expand") - .exclude("Non-vectorized reader - with partition data column - select nested field in Expand") - .exclude("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") - .exclude("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") - .exclude("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") - .exclude("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") - .exclude("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") - .exclude("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") - .exclude("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") - .exclude("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") - .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") - .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") - .exclude( + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( "Case-insensitive parser - mixed-case schema - select with different-case column names") - .exclude( + .excludeCH( "Case-insensitive parser - mixed-case schema - filter with different-case column names") - .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") - .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") - .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") - .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") - .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") - .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") - .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") - .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") - .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") - .exclude("SPARK-36352: Spark should check result plan's output schema name") - .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") - .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") - .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") - .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") - .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") - .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") - .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") - .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") - .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") - .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") - .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") - .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") - .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") - .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") - .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") - .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") - .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH( + "Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .includeCH("aggregate push down - different data types") enableSuite[GlutenParquetV2FilterSuite] - .exclude("filter pushdown - date") + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") .exclude("Filters should be pushed down for Parquet readers at row group level") - .exclude("filter pushdown - StringStartsWith") .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") - .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") - .exclude("SPARK-38825: in and notIn filters") - .exclude("SPARK-36866: filter pushdown - year-month interval") - .excludeGlutenTest("SPARK-25207: exception when duplicate fields in case-insensitive mode") - .excludeGlutenTest("filter pushdown - date") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + .excludeCH("Gluten - filter pushdown - date") enableSuite[GlutenParquetV2PartitionDiscoverySuite] - .exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping") - .exclude("Various partition value types") - .exclude("Various inferred partition value types") - .exclude( - "SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") - .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") enableSuite[GlutenParquetV2QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") - .exclude("SPARK-34212 Parquet should read decimals correctly") enableSuite[GlutenParquetV2SchemaPruningSuite] - .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") - .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") - .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") - .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") - .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") - .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") - .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") - .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") - .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") - .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") - .exclude( + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( "Case-insensitive parser - mixed-case schema - select with different-case column names") - .exclude( + .excludeCH( "Case-insensitive parser - mixed-case schema - filter with different-case column names") - .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") - .exclude("SPARK-36352: Spark should check result plan's output schema name") - .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") - .exclude("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") enableSuite[GlutenParquetVectorizedSuite] - enableSuite[GlutenTextV1Suite] - enableSuite[GlutenTextV2Suite] - enableSuite[GlutenDataSourceV2StrategySuite] - enableSuite[GlutenFileTableSuite] - enableSuite[GlutenV2PredicateSuite] - enableSuite[GlutenEnsureRequirementsSuite] - .exclude("reorder should handle PartitioningCollection") - .exclude("SPARK-35675: EnsureRequirements remove shuffle should respect PartitioningCollection") - enableSuite[GlutenBroadcastJoinSuite] - .exclude("unsafe broadcast hash join updates peak execution memory") - .exclude("unsafe broadcast hash outer join updates peak execution memory") - .exclude("unsafe broadcast left semi join updates peak execution memory") - .exclude("SPARK-23192: broadcast hint should be retained after using the cached data") - .exclude("SPARK-23214: cached data should not carry extra hint info") - .exclude("broadcast hint in SQL") - .exclude("Broadcast timeout") - .exclude("broadcast join where streamed side's output partitioning is HashPartitioning") - .exclude("broadcast join where streamed side's output partitioning is PartitioningCollection") - .exclude("BroadcastHashJoinExec output partitioning size should be limited with a config") - .exclude("SPARK-37742: join planning shouldn't read invalid InMemoryRelation stats") - enableSuite[GlutenExistenceJoinSuite] - .exclude("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build left") - .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") - .exclude("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build left") - .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") - .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") - .exclude("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen on)") - .exclude("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") - .exclude("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") - enableSuite[GlutenInnerJoinSuiteForceShjOn] - .exclude( - "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") - .exclude( - "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") - .exclude( - "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") - .exclude( - "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") - .exclude("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") - .exclude("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") - .exclude( - "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") - .exclude( - "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") - .exclude( - "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") - .exclude( - "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") - .exclude("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") - .exclude("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") - .exclude("inner join, no matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") - .exclude("inner join, no matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") - .exclude( - "inner join, no matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") - .exclude("inner join, no matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") - .exclude("inner join, no matches using SortMergeJoin (whole-stage-codegen off)") - .exclude("inner join, no matches using SortMergeJoin (whole-stage-codegen on)") - .exclude("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") - .exclude("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") - .exclude("inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") - .exclude("inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") - .exclude("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") - .exclude("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using CartesianProduct") - .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") - .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") - .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") - enableSuite[GlutenOuterJoinSuiteForceShjOn] - .exclude("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("basic left outer join using SortMergeJoin (whole-stage-codegen off)") - .exclude("basic left outer join using SortMergeJoin (whole-stage-codegen on)") - .exclude("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("basic right outer join using SortMergeJoin (whole-stage-codegen off)") - .exclude("basic right outer join using SortMergeJoin (whole-stage-codegen on)") - .exclude("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("basic full outer join using SortMergeJoin (whole-stage-codegen off)") - .exclude("basic full outer join using SortMergeJoin (whole-stage-codegen on)") - .exclude("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") - .exclude("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") - .exclude("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") - .exclude("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") - .exclude("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") - .exclude("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") - .exclude("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") - .exclude("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") - enableSuite[GlutenCustomerExtensionSuite] - enableSuite[GlutenSessionExtensionSuite] - enableSuite[GlutenFallbackSuite] - enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] - .exclude("avoid shuffle when join 2 bucketed tables") - .exclude("only shuffle one side when join bucketed table and non-bucketed table") - .exclude("only shuffle one side when 2 bucketed tables have different bucket number") - .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") - .exclude("shuffle when join keys are not equal to bucket keys") - .exclude("shuffle when join 2 bucketed tables with bucketing disabled") - .exclude("check sort and shuffle when bucket and sort columns are join keys") - .exclude("avoid shuffle and sort when sort columns are a super set of join keys") - .exclude("only sort one side when sort columns are different") - .exclude("only sort one side when sort columns are same but their ordering is different") - .exclude("SPARK-17698 Join predicates should not contain filter clauses") - .exclude( - "SPARK-19122 Re-order join predicates if they match with the child's output partitioning") - .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's partitioning columns") - .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") - .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") - .exclude("bucket coalescing eliminates shuffle") - .exclude("bucket coalescing is not satisfied") - // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type - .exclude("disable bucketing when the output doesn't contain all bucketing columns") - .exclude( - "bucket coalescing is applied when join expressions match with partitioning expressions") - enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] - enableSuite[GlutenCreateTableAsSelectSuite] - .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") - .exclude("create a table, drop it and create another one with the same name") - enableSuite[GlutenDDLSourceLoadSuite] - enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] - .disable( - "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") - enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] - enableSuite[GlutenExternalCommandRunnerSuite] - enableSuite[GlutenFilteredScanSuite] - enableSuite[GlutenFiltersSuite] - enableSuite[GlutenInsertSuite] - .excludeAllGlutenTests() enableSuite[GlutenPartitionedWriteSuite] - .exclude("SPARK-37231, SPARK-37240: Dynamic writes/reads of ANSI interval partitions") + .excludeCH("SPARK-37231, SPARK-37240: Dynamic writes/reads of ANSI interval partitions") + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPredicateSuite] + .excludeCH("basic IN/INSET predicate test") + .excludeCH("IN with different types") + .excludeCH("IN/INSET: binary") + .excludeCH("IN/INSET: struct") + .excludeCH("IN/INSET: array") + .excludeCH("BinaryComparison: lessThan") + .excludeCH("BinaryComparison: LessThanOrEqual") + .excludeCH("BinaryComparison: GreaterThan") + .excludeCH("BinaryComparison: GreaterThanOrEqual") + .excludeCH("EqualTo on complex type") + .excludeCH("SPARK-32764: compare special double/float values") + .excludeCH("SPARK-32110: compare special double/float values in struct") + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + .excludeCH("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") + .excludeCH( + "ALTER NAMESPACE with property other than COMMENT for JDBC catalog should throw an exception") + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + // Doesn't support unhex with failOnError=true. + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function hex") + .excludeCH("CONVERSION_INVALID_INPUT: to_binary conversion function base64") + .excludeCH("UNSUPPORTED_FEATURE - SPARK-38504: can't read TimestampNTZ as TimestampLTZ") + .excludeCH("CANNOT_PARSE_DECIMAL: unparseable decimal") + .excludeCH("UNRECOGNIZED_SQL_TYPE: unrecognized SQL type DATALINK") + .excludeCH("UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER: The target JDBC server hosting table does not support ALTER TABLE with multiple actions.") + .excludeCH("INVALID_BITMAP_POSITION: position out of bounds") + .excludeCH("INVALID_BITMAP_POSITION: negative position") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + .excludeCH("LIKE Pattern") + .excludeCH("LIKE Pattern ESCAPE '/'") + .excludeCH("LIKE Pattern ESCAPE '#'") + .excludeCH("LIKE Pattern ESCAPE '\"'") + .excludeCH("RLIKE Regular Expression") + .excludeCH("RegexReplace") + .excludeCH("RegexExtract") + .excludeCH("RegexExtractAll") + .excludeCH("SPLIT") + enableSuite[GlutenRemoveRedundantWindowGroupLimitsSuite] + .excludeCH("remove redundant WindowGroupLimits") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .exclude("do not replace hash aggregate if child does not have sort order") + .exclude("do not replace hash aggregate if there is no group-by column") + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenResolveDefaultColumnsSuite] enableSuite[GlutenResolvedDataSourceSuite] - enableSuite[GlutenSaveLoadSuite] - enableSuite[GlutenTableScanSuite] - .exclude("Schema and all fields") - .exclude("SELECT count(*) FROM tableWithSchema") - .exclude("SELECT `string$%Field` FROM tableWithSchema") - .exclude("SELECT int_Field FROM tableWithSchema WHERE int_Field < 5") - .exclude("SELECT `longField_:,<>=+/~^` * 2 FROM tableWithSchema") + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenRuntimeNullChecksV2Writes] + enableSuite[GlutenSQLAggregateFunctionSuite] + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .includeCH("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .includeCH("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .includeCH("SPARK-28156: self-join should not miss cached view") + .includeCH("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .includeCH("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Not useful and time consuming. + .includeCH("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + // exception test, rewritten in gluten + .exclude("the escape character is not allowed to end with") + // ORC related + .includeCH("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .includeCH("SPARK-38173: Quoted column cannot be recognized correctly when quotedRegexColumnNames is true") + // Need to support MAP .exclude( - "SELECT structFieldSimple.key, arrayFieldSimple[1] FROM tableWithSchema a where int_Field=1") - .exclude("SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema") - enableSuite[SparkFunctionStatistics] + "SPARK-27619: When spark.sql.legacy.allowHashOnMapType is true, hash can be used on Maptype") + .excludeCH("SPARK-6743: no columns from cache") + .excludeCH("external sorting updates peak execution memory") + .excludeCH("Struct Star Expansion") + .excludeCH("Common subexpression elimination") + .excludeCH("SPARK-24940: coalesce and repartition hint") + .excludeCH("normalize special floating numbers in subquery") + .excludeCH("SPARK-38548: try_sum should return null if overflow happens before merging") + .excludeCH("SPARK-38589: try_avg should return null if overflow happens before merging") + .excludeCH("Gluten - SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + .excludeCH("Gluten - the escape character is not allowed to end with") + enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenSQLWindowFunctionSuite] + // spill not supported yet. + .exclude("test with low buffer spill threshold") + .excludeCH( + "window function: multiple window expressions specified by range in a single expression") + .excludeCH("Gluten - Filter on row number") + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenSessionExtensionSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + enableSuite[GlutenSortShuffleSuite] + enableSuite[GlutenSortSuite] + .excludeCH("basic sorting using ExternalSort") + .excludeCH("SPARK-33260: sort order is a Stream") + .excludeCH("SPARK-40089: decimal values sort correctly") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") enableSuite[GlutenSparkSessionExtensionSuite] - enableSuite[GlutenHiveSQLQueryCHSuite] - enableSuite[GlutenPercentileSuite] - enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] - enableSuite[GlutenParquetRowIndexSuite] - .excludeByPrefix("row index generation") - .excludeByPrefix("invalid row index column type") + enableSuite[GlutenStatisticsCollectionSuite] + // The output byte size of Velox is different + .includeCH("SPARK-33687: analyze all tables in a specific database") + .excludeCH("analyze empty table") + .excludeCH("analyze column command - result verification") + .excludeCH("column stats collection for null columns") + .excludeCH("store and retrieve column stats in different time zones") + .excludeCH("SPARK-42777: describe column stats (min, max) for timestamp_ntz column") + .excludeCH("Gluten - store and retrieve column stats in different time zones") + enableSuite[GlutenStringExpressionsSuite] + .excludeCH("StringComparison") + .excludeCH("Substring") + .excludeCH("string substring_index function") + .excludeCH("SPARK-40213: ascii for Latin-1 Supplement characters") + .excludeCH("ascii for string") + .excludeCH("Mask") + .excludeCH("SPARK-42384: Mask with null input") + .excludeCH("base64/unbase64 for string") + .excludeCH("encode/decode for string") + .excludeCH("SPARK-47307: base64 encoding without chunking") + .excludeCH("Levenshtein distance threshold") + .excludeCH("soundex unit test") + .excludeCH("overlay for string") + .excludeCH("overlay for byte array") + .excludeCH("translate") + .excludeCH("FORMAT") + .excludeCH("LOCATE") + .excludeCH("REPEAT") + .excludeCH("ParseUrl") + .excludeCH("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") + enableSuite[GlutenStringFunctionsSuite] + .excludeCH("string Levenshtein distance") + .excludeCH("string regexp_count") + .excludeCH("string regex_replace / regex_extract") + .excludeCH("string regexp_extract_all") + .excludeCH("string regexp_substr") + .excludeCH("string overlay function") + .excludeCH("binary overlay function") + .excludeCH("string / binary length function") + .excludeCH("SPARK-36751: add octet length api for scala") + .excludeCH("SPARK-36751: add bit length api for scala") + .excludeCH("str_to_map function") + .excludeCH("SPARK-42384: mask with null input") + .excludeCH("like & ilike function") + .excludeCH("parse_url") + .excludeCH("url_decode") + .excludeCH("url_encode") + enableSuite[GlutenSubqueryHintPropagationSuite] + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + .excludeCH("SPARK-39355: Single column uses quoted to construct UnresolvedAttribute") + .excludeCH("SPARK-40800: always inline expressions in OptimizeOneRowRelationSubquery") + .excludeCH("SPARK-40862: correlated one-row subquery with non-deterministic expressions") + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenTableLocationSuite] + enableSuite[GlutenTableOptionsConstantFoldingSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenTakeOrderedAndProjectSuite] + .excludeCH("TakeOrderedAndProject.doExecute without project") + .excludeCH("TakeOrderedAndProject.doExecute with project") + .excludeCH("TakeOrderedAndProject.doExecute with local sort") + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenTryCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + .exclude("ANSI mode: Throw exception on casting out-of-range value to byte type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to short type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to int type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to long type") + .exclude("cast from invalid string to numeric should throw NumberFormatException") + .exclude("SPARK-26218: Fix the corner case of codegen when casting float to Integer") + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("cast from timestamp II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .excludeCH("cast from array II") + .excludeCH("cast from array III") + .excludeCH("cast from struct III") + .excludeCH("ANSI mode: cast string to timestamp with parse error") + .excludeCH("ANSI mode: cast string to date with parse error") + .excludeCH("Gluten - data type casting") + enableSuite[GlutenTryEvalSuite] + .excludeCH("try_subtract") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + // Rewrite with NaN test cases excluded. + .exclude("cases when literal is max") + enableSuite[GlutenUrlFunctionsSuite] + .excludeCH("url parse_url function") + .excludeCH("url encode/decode function") + enableSuite[GlutenV1WriteCommandSuite] + // Rewrite to match SortExecTransformer. + .excludeByPrefix("SPARK-41914:") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition column") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - string partition column") + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + // Rewrite to disable Spark's vectorized reading. + .includeCH("change column position") + .includeCH("read byte, int, short, long together") + .includeCH("read float and double together") + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("change column type from float to double") + // .excludeGlutenTest("read byte, int, short, long together") + // .excludeGlutenTest("read float and double together") + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenXPathFunctionsSuite] override def getSQLQueryTestSettings: SQLQueryTestSettings = ClickHouseSQLQueryTestSettings } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala index b99c2aef6cb9..44a370cbc374 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala @@ -20,4 +20,4 @@ class GlutenFileSourceSQLInsertTestSuite extends FileSourceSQLInsertTestSuite with GlutenSQLTestsTrait {} -class GlutenDSV2SQLInsertTestSuite extends DSV2SQLInsertTestSuite {} +class GlutenDSV2SQLInsertTestSuite extends DSV2SQLInsertTestSuite with GlutenTestsBaseTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala index bd9699e4008c..8f397c517ef1 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala @@ -129,7 +129,7 @@ class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLTestsTrait { } } - testGluten("StreamingQueryProgress.numInputRows should be correct") { + ignoreGluten("StreamingQueryProgress.numInputRows should be correct") { withTempDir { dir => val path = dir.toURI.getPath diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 5810cc5abe76..94558949731d 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -227,7 +227,8 @@ class GlutenSQLQueryTestSuite // Disable for Spark 3. "group-by.sql", "udf/udf-group-by.sql - Scala UDF" - ) ++ otherIgnoreList ++ udafIgnoreList + ) ++ otherIgnoreList ++ udafIgnoreList ++ + BackendTestSettings.instance.getSQLQueryTestSettings.getIgnoredSQLQueryTests // List of supported cases to run with a certain backend, in lower case. private val supportedList: Set[String] = diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index a8eb2361ab3e..739343587e28 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -44,8 +44,8 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { val testFileSourceScanExecTransformer = df.queryExecution.executedPlan.collect { case f: TestFileSourceScanExecTransformer => f } - assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) + assert(testFileSourceScanExecTransformer.nonEmpty) + assert(testFileSourceScanExecTransformer.head.nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala index 6365305140e5..5ee5cdf20234 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveUDFSuite.scala @@ -81,9 +81,9 @@ class GlutenHiveUDFSuite with GlutenTestHiveSingleton with SQLTestUtils with GlutenTestsBaseTrait { - override protected val spark: SparkSession = GlutenTestHive.sparkSession - protected val hiveContext: TestHiveContext = GlutenTestHive - protected val hiveClient: HiveClient = + override protected lazy val spark: SparkSession = GlutenTestHive.sparkSession + protected lazy val hiveContext: TestHiveContext = GlutenTestHive + protected lazy val hiveClient: HiveClient = spark.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client override protected def beforeAll(): Unit = {