Skip to content

Commit

Permalink
Normalize benchmark file path
Browse files Browse the repository at this point in the history
  • Loading branch information
baibaichen committed Jan 28, 2025
1 parent aeef94f commit 4552a46
Show file tree
Hide file tree
Showing 11 changed files with 49 additions and 41 deletions.
7 changes: 3 additions & 4 deletions cpp-ch/local-engine/tests/benchmark_local_engine.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,7 @@ DB::ContextMutablePtr global_context;
{
substrait::ReadRel::LocalFiles files;
substrait::ReadRel::LocalFiles::FileOrFiles * file = files.add_items();
std::string file_path = "file:///home/hongbin/code/gluten/jvm/src/test/resources/tpch-data/lineitem/"
"part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet";
std::string file_path{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};
file->set_uri_file(file_path);
substrait::ReadRel::LocalFiles::FileOrFiles::ParquetReadOptions parquet_format;
file->mutable_parquet()->CopyFrom(parquet_format);
Expand Down Expand Up @@ -846,8 +845,8 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St
}
auto hash_join = std::make_shared<HashJoin>(join, right->getCurrentHeader());

QueryPlanStepPtr join_step
= std::make_unique<JoinStep>(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false);
QueryPlanStepPtr join_step = std::make_unique<JoinStep>(
left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false);

std::vector<QueryPlanPtr> plans;
plans.emplace_back(std::move(left));
Expand Down
38 changes: 22 additions & 16 deletions cpp-ch/local-engine/tests/benchmark_parquet_read.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
#include <Processors/Formats/Impl/ParquetBlockInputFormat.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/Parquet/ParquetMeta.h>
#include <Storages/Parquet/VectorizedParquetRecordReader.h>
#include <Storages/SubstraitSource/SubstraitFileSource.h>
#include <benchmark/benchmark.h>
#include <parquet/arrow/reader.h>
Expand All @@ -39,15 +41,22 @@ void BM_ColumnIndexRead_NoFilter(benchmark::State & state)
{
using namespace DB;

std::string file = "/home/chang/test/tpch/parquet/s100/lineitem1/"
"part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet";
std::string file = local_engine::test::third_party_data(
"benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet");
Block header{toBlockRowType(local_engine::test::readParquetSchema(file))};
FormatSettings format_settings;
Block res;
for (auto _ : state)
{
auto in = std::make_unique<ReadBufferFromFile>(file);
auto format = std::make_shared<local_engine ::VectorizedParquetBlockInputFormat>(*in, header, format_settings);
local_engine::ParquetMetaBuilder metaBuilder{
.collectPageIndex = true,
.collectSkipRowGroup = false,
.case_insensitive = format_settings.parquet.case_insensitive_column_matching,
.allow_missing_columns = format_settings.parquet.allow_missing_columns};
ReadBufferFromFilePRead fileReader(file);
metaBuilder.build(&fileReader, &header, nullptr, [](UInt64 /*midpoint_offset*/) -> bool { return true; });
local_engine::ColumnIndexRowRangesProvider provider{metaBuilder};
auto format = std::make_shared<local_engine ::VectorizedParquetBlockInputFormat>(fileReader, header, provider, format_settings);
auto pipeline = QueryPipeline(std::move(format));
auto reader = std::make_unique<PullingPipelineExecutor>(pipeline);
while (reader->pull(res))
Expand All @@ -61,15 +70,15 @@ void BM_ColumnIndexRead_Old(benchmark::State & state)
{
using namespace DB;

std::string file = "/home/chang/test/tpch/parquet/s100/lineitem1/"
"part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet";
std::string file = local_engine::test::third_party_data(
"benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet");
Block header{toBlockRowType(local_engine::test::readParquetSchema(file))};
FormatSettings format_settings;
Block res;
for (auto _ : state)
{
auto in = std::make_unique<ReadBufferFromFile>(file);
auto format = std::make_shared<ParquetBlockInputFormat>(*in, header, format_settings, 1, 1, 8192);
ReadBufferFromFilePRead fileReader(file);
auto format = std::make_shared<ParquetBlockInputFormat>(fileReader, header, format_settings, 1, 1, 8192);
auto pipeline = QueryPipeline(std::move(format));
auto reader = std::make_unique<PullingPipelineExecutor>(pipeline);
while (reader->pull(res))
Expand All @@ -86,8 +95,7 @@ void BM_ParquetReadDate32(benchmark::State & state)
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_shipdate"),
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_commitdate"),
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_receiptdate")};
std::string file = "/data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/"
"part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet";
std::string file{GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};
FormatSettings format_settings;
Block res;
for (auto _ : state)
Expand All @@ -110,8 +118,7 @@ void BM_OptimizedParquetReadString(benchmark::State & state)
Block header{
ColumnWithTypeAndName(DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "l_returnflag"),
ColumnWithTypeAndName(DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "l_linestatus")};
std::string file = "file:///data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/"
"part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet";
std::string file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};
Block res;

for (auto _ : state)
Expand Down Expand Up @@ -142,8 +149,7 @@ void BM_OptimizedParquetReadDate32(benchmark::State & state)
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_shipdate"),
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_commitdate"),
ColumnWithTypeAndName(DataTypeDate32().createColumn(), std::make_shared<DataTypeDate32>(), "l_receiptdate")};
std::string file = "file:///data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/"
"part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet";
std::string file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};
Block res;

for (auto _ : state)
Expand Down Expand Up @@ -209,7 +215,7 @@ void BM_ColumnIndexRead_Filter_ReturnAllResult(benchmark::State & state)
{
using namespace DB;

const std::string filename = local_engine::test::third_party_data_file(
const std::string filename = local_engine::test::third_party_data(
"benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet");
const std::string filter1 = "l_shipdate is not null AND l_shipdate <= toDate32('1998-09-01')";
const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true);
Expand All @@ -226,7 +232,7 @@ void BM_ColumnIndexRead_Filter_ReturnHalfResult(benchmark::State & state)
{
using namespace DB;

const std::string filename = local_engine::test::third_party_data_file(
const std::string filename = local_engine::test::third_party_data(
"benchmark/column_index/lineitem/part-00000-9395e12a-3620-4085-9677-c63b920353f4-c000.snappy.parquet");
const std::string filter1 = "l_orderkey is not null AND l_orderkey > 300977829";
const substrait::ReadRel::LocalFiles files = createLocalFiles(filename, true);
Expand Down
4 changes: 2 additions & 2 deletions cpp-ch/local-engine/tests/gluten_test_util.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ std::string internal_data_file(const char * file, const std::string & dir_string
}

/// used with the PARQUET_TEST_DATA environment variable
std::string third_party_data_file(const char * file)
std::string third_party_data(const char * file)
{
return internal_data_file(file, get_data_dir());
}
Expand All @@ -130,7 +130,7 @@ std::string gtest_data(const char * file)
}

/// It represents a test data file in 'utils/extern-local-engine/tests/data' with 'file://' schema
std::string gtest_file(const char * file)
std::string gtest_uri(const char * file)
{
#define GLUTEN_DATA_DIR(file) "file://" SOURCE_DIR file
return internal_data_file(file, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data"));
Expand Down
6 changes: 3 additions & 3 deletions cpp-ch/local-engine/tests/gluten_test_util.h
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,9 @@ class RandomAccessFile;
namespace local_engine::test
{

std::string third_party_data_file(const char * file);
std::string third_party_data(const char * file);
std::string gtest_data(const char * file);
std::string gtest_file(const char * file);
std::string gtest_uri(const char * file);

std::shared_ptr<arrow::io::RandomAccessFile> asArrowFileForParquet(DB::ReadBuffer & in, const DB::FormatSettings & settings);

Expand Down Expand Up @@ -93,7 +93,7 @@ inline std::string replaceLocalFilesWildcards(const std::string_view haystack, c
inline std::string replaceLocalFilesWithTPCH(const std::string_view haystack)
{
static constexpr auto wildcard = "{replace_GLUTEN_SOURCE_TPCH_DIR}";
constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_DIR("");
constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_URI("");
return boost::replace_all_copy(std::string{haystack}, wildcard, replaced);
}

Expand Down
6 changes: 3 additions & 3 deletions cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ TEST(Clickhouse, PR54881)
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"1529","parquet":{},"schema":{},"metadataColumns":[{}]}]})";

const std::string file{test::gtest_file("54881.snappy.parquet")};
const std::string file{test::gtest_uri("54881.snappy.parquet")};
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_54881_), split_template, file, context1);

EXPECT_TRUE(local_executor->hasNext());
Expand Down Expand Up @@ -102,7 +102,7 @@ TEST(Clickhouse, PR68135)
{
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"461","parquet":{},"schema":{},"metadataColumns":[{}]}]})";
const std::string file{test::gtest_file("68135.snappy.parquet")};
const std::string file{test::gtest_uri("68135.snappy.parquet")};
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68135_), split_template, file);

EXPECT_TRUE(local_executor->hasNext());
Expand All @@ -115,7 +115,7 @@ TEST(Clickhouse, PR68131)
{
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"289","parquet":{},"schema":{},"metadataColumns":[{}]}]})";
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68131_), split_template, test::gtest_file("68131.parquet"));
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68131_), split_template, test::gtest_uri("68131.parquet"));
EXPECT_TRUE(local_executor->hasNext());
const Block & x = *local_executor->nextColumnar();
debug::headBlock(x);
Expand Down
10 changes: 5 additions & 5 deletions cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ TEST(ColumnIndex, Decimal182)

constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"488","parquet":{},"schema":{},"metadataColumns":[{}]}]})";
const std::string file{test::gtest_file("decimal_filter_push_down/18_2_flba.snappy.parquet")};
const std::string file{test::gtest_uri("decimal_filter_push_down/18_2_flba.snappy.parquet")};
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_18_2), split_template, file, context);

EXPECT_TRUE(local_executor->hasNext());
Expand All @@ -69,7 +69,7 @@ void readFile(
INCBIN(_read_metadata, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/read_metadata.row_index.json");
TEST(RowIndex, Basic)
{
const std::string file{test::gtest_file("metadata.rowindex.snappy.parquet")};
const std::string file{test::gtest_uri("metadata.rowindex.snappy.parquet")};
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"1767","parquet":{},"partitionColumns":[{"key":"pb","value":"1003"}],"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"1767"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"1767","modificationTime":"1736847651881"}}]})";
readFile(
Expand All @@ -86,7 +86,7 @@ TEST(RowIndex, Basic)
INCBIN(_rowindex_in, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/rowindex_in.json");
TEST(RowIndex, In)
{
const std::string file{test::gtest_file("rowindex_in.snappy.parquet")};
const std::string file{test::gtest_uri("rowindex_in.snappy.parquet")};
/// all row gorups are ignored
constexpr std::string_view split_template_ignore_all_rg
= R"({"items":[{"uriFile":"{replace_local_files}","length":"256","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"256"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"125451","modificationTime":"1737104830724"}}]})";
Expand Down Expand Up @@ -114,7 +114,7 @@ TEST(RowIndex, In)
INCBIN(_all_meta, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/read_metadata.all.json");
TEST(RowIndex, AllMeta)
{
const std::string file{test::gtest_file("all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet")};
const std::string file{test::gtest_uri("all_meta/part-00000-92bb25d0-7446-4f9b-8bdd-a6911d0d465a-c000.snappy.parquet")};
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"1282","parquet":{},"schema":{},"metadataColumns":[{"key":"file_path","value":"{replace_local_files}"},{"key":"file_block_length","value":"1282"},{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"1282"},{"key":"file_name","value":"part-00000-484a7344-cf25-4367-bf46-8123a6a7b71e-c000.snappy.parquet"},{"key":"file_modification_time","value":"2025-01-19 05:09:48.664"},{"key":"file_block_start","value":"0"},{"key":"input_file_block_start","value":"0"},{"key":"file_size","value":"1282"}],"properties":{"fileSize":"1282","modificationTime":"1737263388664"}}]})";

Expand All @@ -134,7 +134,7 @@ INCBIN(
_input_filename_no_real_column, SOURCE_DIR "/utils/extern-local-engine/tests/json/parquet_metadata/input_filename_no_real_column.json");
TEST(RowIndex, InputFileName)
{
const std::string file{test::gtest_file("input_filename.snappy.parquet")};
const std::string file{test::gtest_uri("input_filename.snappy.parquet")};
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"443","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"443"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"443","modificationTime":"1737445386987"}}]})";

Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/tests/gtest_parquet_read.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -453,7 +453,7 @@ TEST(ParquetRead, UpperColRead)
{
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"459","parquet":{},"schema":{},"metadataColumns":[{"key":"input_file_name","value":"{replace_local_files}"},{"key":"input_file_block_length","value":"459"},{"key":"input_file_block_start","value":"0"}],"properties":{"fileSize":"459","modificationTime":"1735012863732"}}]})";
const std::string file{test::gtest_file("upper_case_col.parquet")};
const std::string file{test::gtest_uri("upper_case_col.parquet")};
auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_upper_col_parquet_), split_template, file, {});
EXPECT_TRUE(local_executor->hasNext());
const Block & block = *local_executor->nextColumnar();
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/tests/gtest_parser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ TEST(LocalExecutor, ReadCSV)
constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"56","text":{"fieldDelimiter":",","maxBlockSize":"8192","header":"1"},"schema":{"names":["id","name","language"],"struct":{"types":[{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})";
const std::string split = replaceLocalFilesWildcards(
split_template, GLUTEN_SOURCE_DIR("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv"));
split_template, GLUTEN_SOURCE_URI("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv"));
auto plan = local_engine::JsonStringToMessage<substrait::Plan>(EMBEDDED_PLAN(_readcsv_plan));
auto parser_context = ParserContext::build(QueryContext::globalContext(), plan);
SerializedPlanParser parser(parser_context);
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/tests/gtest_write_pipeline.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ TEST(WritePipeline, SubstraitFileSink)

constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})";
constexpr std::string_view file{GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")};
constexpr std::string_view file{GLUTEN_SOURCE_URI("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")};
auto [plan, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(native_write), split_template, file, context);

EXPECT_EQ(1, plan.relations_size());
Expand Down
5 changes: 2 additions & 3 deletions cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,7 @@ TEST(MergeTree, SparkMergeTree)

constexpr std::string_view split_template
= R"({"items":[{"uriFile":"{replace_local_files}","length":"19230111","parquet":{},"schema":{},"metadataColumns":[{}],"properties":{"fileSize":"19230111","modificationTime":"1722330598029"}}]})";
constexpr std::string_view file{GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};
constexpr std::string_view file{GLUTEN_SOURCE_TPCH_URI("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")};

SparkMergeTreeWritePartitionSettings gm_write_settings{
.part_name_prefix{"this_is_prefix"},
Expand Down Expand Up @@ -293,8 +293,7 @@ TEST(MergeTree, Pipeline)
writeMerge(
EMBEDDED_PLAN(_3_mergetree_plan_),
"tmp/lineitem_mergetree",
{{"min_insert_block_size_rows", 100000}
/*, {"optimize.minFileSize", 1024 * 1024 * 10}*/},
{{"min_insert_block_size_rows", 100000} /*, {"optimize.minFileSize", 1024 * 1024 * 10}*/},
[&](const DB::Block & block)
{
EXPECT_EQ(1, block.rows());
Expand Down
8 changes: 6 additions & 2 deletions cpp-ch/local-engine/tests/testConfig.h.in
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,10 @@
#define PARQUET_DATA(file) "file://@PARQUET_DATA_DIR@"#file
#define MERGETREE_DATA(file) "@MERGETREE_DATA_DIR@"#file

#define GLUTEN_SOURCE_DIR_ "file://@GLUTEN_REAL_PATH@"
#define GLUTEN_SOURCE_URI_ "file://@GLUTEN_REAL_PATH@"
#define GLUTEN_SOURCE_URI(file) GLUTEN_SOURCE_URI_ file
#define GLUTEN_SOURCE_TPCH_URI(file) GLUTEN_SOURCE_URI("/gluten-core/src/test/resources/tpch-data/" file)

#define GLUTEN_SOURCE_DIR_ "@GLUTEN_REAL_PATH@"
#define GLUTEN_SOURCE_DIR(file) GLUTEN_SOURCE_DIR_ file
#define GLUTEN_SOURCE_TPCH_DIR(file) GLUTEN_SOURCE_DIR("/gluten-core/src/test/resources/tpch-data/" file)
#define GLUTEN_SOURCE_TPCH_DIR(file) GLUTEN_SOURCE_DIR("/gluten-core/src/test/resources/tpch-data/" file)

0 comments on commit 4552a46

Please sign in to comment.