diff --git a/integration_tests/src/main/python/parquet_test.py b/integration_tests/src/main/python/parquet_test.py index c41e025b050..9faf5ed461f 100644 --- a/integration_tests/src/main/python/parquet_test.py +++ b/integration_tests/src/main/python/parquet_test.py @@ -477,3 +477,50 @@ def _create_wide_data_frame(spark, num_cols): assert_gpu_and_cpu_are_equal_collect( func=lambda spark: _create_wide_data_frame(spark, 1000), is_cpu_first=False) + +def setup_parquet_file_with_column_names(spark, table_name): + drop_query = "DROP TABLE IF EXISTS {}".format(table_name) + create_query = "CREATE TABLE `{}` (`a` INT, `b` ARRAY, `c` STRUCT<`c_1`: INT, `c_2`: STRING>) USING parquet"\ + .format(table_name) + insert_query = "INSERT INTO {} VALUES(13, array(2020), named_struct('c_1', 1, 'c_2', 'hello'))".format(table_name) + spark.sql(drop_query).collect + spark.sql(create_query).collect + spark.sql(insert_query).collect + +@pytest.mark.parametrize('reader_confs', reader_opt_confs, ids=idfn) +@pytest.mark.parametrize('v1_enabled_list', ["", "parquet"]) +def test_disorder_read_schema_XXX(spark_tmp_table_factory, reader_confs, v1_enabled_list): + all_confs = reader_confs.copy() + all_confs.update({'spark.sql.sources.useV1SourceList': v1_enabled_list}) + table_name = spark_tmp_table_factory.get() + with_cpu_session(lambda spark : setup_parquet_file_with_column_names(spark, table_name)) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT a,b FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT c,a FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT c,b FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT a,c,b FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT a,b,c FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT b,c,a FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT b,c,a FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT c,a,b FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT c,b,a FROM {}".format(table_name)), + all_confs) + assert_gpu_and_cpu_are_equal_collect( + lambda spark : spark.sql("SELECT c.c_2,c.c_1,b,a FROM {}".format(table_name)), + all_confs)