Skip to content

Commit

Permalink
Fix UT build due to #7193
Browse files Browse the repository at this point in the history
  • Loading branch information
baibaichen committed Sep 13, 2024
1 parent 029c865 commit f8c4599
Show file tree
Hide file tree
Showing 17 changed files with 37 additions and 42 deletions.
2 changes: 0 additions & 2 deletions cpp-ch/local-engine/Common/CHUtil.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
#include <memory>
#include <optional>
#include <unistd.h>

#include <AggregateFunctions/Combinators/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Columns/ColumnArray.h>
Expand Down Expand Up @@ -52,7 +51,6 @@
#include <Parser/SubstraitParserUtils.h>
#include <Planner/PlannerActionsVisitor.h>
#include <Processors/Chunk.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/Parser/LocalExecutor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
#include "LocalExecutor.h"

#include <Core/Settings.h>
#include <Parser/SerializedPlanParser.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <QueryPipeline/printPipeline.h>
#include <Common/QueryContext.h>
Expand Down
1 change: 0 additions & 1 deletion cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
#include <Join/BroadCastJoinBuilder.h>
#include <Join/StorageJoinFromReadBuffer.h>
#include <Parser/AdvancedParametersParseUtil.h>
#include <Parser/SerializedPlanParser.h>
#include <Parsers/ASTIdentifier.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
Expand Down
1 change: 0 additions & 1 deletion cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
#include <Operator/ExpandStep.h>
#include <Parser/ExpandField.h>
#include <Parser/RelParsers/RelParser.h>
#include <Parser/SerializedPlanParser.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Common/logger_useful.h>

Expand Down
4 changes: 2 additions & 2 deletions cpp-ch/local-engine/Parser/RelParsers/FetchRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,9 @@

#include <memory>
#include <optional>
#include <Parser/SerializedPlanParser.h>
#include <Parser/RelParsers/RelParser.h>
#include <Processors/QueryPlan/LimitStep.h>
#include "RelParser.h"

namespace local_engine
{
class FetchRelParser : public RelParser
Expand Down
1 change: 0 additions & 1 deletion cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
#include <Join/StorageJoinFromReadBuffer.h>
#include <Operator/EarlyStopStep.h>
#include <Parser/AdvancedParametersParseUtil.h>
#include <Parser/SerializedPlanParser.h>
#include <Parsers/ASTIdentifier.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
Expand Down
14 changes: 14 additions & 0 deletions cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,20 @@

#include "ReadRelParser.h"
#include <memory>
#include <Core/Block.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Operator/BlocksBufferPoolTransform.h>
#include <Parser/RelParsers/MergeTreeRelParser.h>
#include <Parser/SubstraitParserUtils.h>
#include <Parser/TypeParser.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Storages/SourceFromJavaIter.h>
#include <Storages/SubstraitSource/SubstraitFileSource.h>
#include <Storages/SubstraitSource/SubstraitFileSourceStep.h>
#include <google/protobuf/wrappers.pb.h>
#include <Common/BlockTypeUtils.h>


namespace DB::ErrorCodes
{
Expand Down
16 changes: 2 additions & 14 deletions cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,24 +15,12 @@
* limitations under the License.
*/
#pragma once
#include <Core/Block.h>
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Operator/BlocksBufferPoolTransform.h>
#include <Parser/RelParsers/MergeTreeRelParser.h>
#include <Parser/SerializedPlanParser.h>
#include <Parser/SubstraitParserUtils.h>
#include <Parser/TypeParser.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Storages/SourceFromJavaIter.h>
#include <Storages/SubstraitSource/SubstraitFileSource.h>
#include <Storages/SubstraitSource/SubstraitFileSourceStep.h>
#include <google/protobuf/wrappers.pb.h>
#include <Common/BlockTypeUtils.h>
#include <Parser/RelParsers/RelParser.h>
#include <Common/JNIUtils.h>

namespace local_engine
{

class ReadRelParser : public RelParser
{
public:
Expand Down
4 changes: 1 addition & 3 deletions cpp-ch/local-engine/Parser/RelParsers/RelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,10 @@
#include "RelParser.h"

#include <string>
#include <google/protobuf/wrappers.pb.h>

#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/IDataType.h>
#include <Poco/Logger.h>
#include <google/protobuf/wrappers.pb.h>
#include <Poco/StringTokenizer.h>
#include <Common/Exception.h>
#include <Common/logger_useful.h>
Expand Down
18 changes: 11 additions & 7 deletions cpp-ch/local-engine/Parser/RelParsers/RelParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
#include <base/types.h>
#include <substrait/extensions/extensions.pb.h>
#include <substrait/plan.pb.h>

namespace local_engine
{
/// parse a single substrait relation
Expand All @@ -52,34 +53,37 @@ class RelParser
inline SerializedPlanParser * getPlanParser() const { return plan_parser; }
inline ContextPtr getContext() const { return plan_parser->context; }

inline String getUniqueName(const std::string & name) { return plan_parser->getUniqueName(name); }
inline String getUniqueName(const std::string & name) const { return plan_parser->getUniqueName(name); }

inline const std::unordered_map<std::string, std::string> & getFunctionMapping() { return plan_parser->function_mapping; }
inline const std::unordered_map<std::string, std::string> & getFunctionMapping() const { return plan_parser->function_mapping; }

// Get function signature name.
std::optional<String> parseSignatureFunctionName(UInt32 function_ref);
// Get coresponding function name in ClickHouse.
std::optional<String> parseFunctionName(UInt32 function_ref, const substrait::Expression_ScalarFunction & function);

const DB::ActionsDAG::Node * parseArgument(ActionsDAG & action_dag, const substrait::Expression & rel)
const DB::ActionsDAG::Node * parseArgument(ActionsDAG & action_dag, const substrait::Expression & rel) const
{
return plan_parser->parseExpression(action_dag, rel);
}

const DB::ActionsDAG::Node * parseExpression(ActionsDAG & action_dag, const substrait::Expression & rel)
const DB::ActionsDAG::Node * parseExpression(ActionsDAG & action_dag, const substrait::Expression & rel) const
{
return plan_parser->parseExpression(action_dag, rel);
}
DB::ActionsDAG expressionsToActionsDAG(const std::vector<substrait::Expression> & expressions, const DB::Block & header)
DB::ActionsDAG expressionsToActionsDAG(const std::vector<substrait::Expression> & expressions, const DB::Block & header) const
{
return plan_parser->expressionsToActionsDAG(expressions, header, header);
}
std::pair<DataTypePtr, Field> parseLiteral(const substrait::Expression_Literal & literal) { return plan_parser->parseLiteral(literal); }
std::pair<DataTypePtr, Field> parseLiteral(const substrait::Expression_Literal & literal) const
{
return plan_parser->parseLiteral(literal);
}
// collect all steps for metrics
std::vector<IQueryPlanStep *> steps;

const ActionsDAG::Node *
buildFunctionNode(ActionsDAG & action_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args)
buildFunctionNode(ActionsDAG & action_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) const
{
return plan_parser->toFunctionNode(action_dag, function, args);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
#pragma once
#include <optional>
#include <Parser/RelParsers/RelParser.h>
#include <Parser/SerializedPlanParser.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Poco/Logger.h>
#include <Common/logger_useful.h>
Expand Down
1 change: 0 additions & 1 deletion cpp-ch/local-engine/Parser/SerializedPlanParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,6 @@
#include <Common/JNIUtils.h>
#include <Common/logger_useful.h>
#include <Common/typeid_cast.h>
#include "RelParsers/RelParser.h"

namespace DB
{
Expand Down
7 changes: 3 additions & 4 deletions cpp-ch/local-engine/Shuffle/PartitionWriter.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
#include <Core/Block.h>
#include <Core/Settings.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Parser/SerializedPlanParser.h>
#include <Shuffle/ShuffleCommon.h>
#include <jni/CelebornClient.h>
#include <Common/GlutenConfig.h>
Expand Down Expand Up @@ -68,7 +67,7 @@ friend class Spillable;
PartitionWriter(const SplitOptions& options, LoggerPtr logger_);
virtual ~PartitionWriter() = default;

void initialize(SplitResult * split_result_, const Block & output_header_)
void initialize(SplitResult * split_result_, const DB::Block & output_header_)
{
if (!init)
{
Expand Down Expand Up @@ -107,8 +106,8 @@ friend class Spillable;

/// Only valid in celeborn partition writer
size_t last_partition_id;
SplitResult* split_result = nullptr;
Block output_header;
SplitResult * split_result = nullptr;
DB::Block output_header;
LoggerPtr logger = nullptr;
bool init = false;
};
Expand Down
1 change: 0 additions & 1 deletion cpp-ch/local-engine/Shuffle/ShuffleCommon.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
*/
#include <Shuffle/ShuffleCommon.h>
#include <Storages/IO/AggregateSerializationUtils.h>
#include <Parser/SerializedPlanParser.h>
#include <Poco/StringTokenizer.h>

namespace local_engine
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/tests/gtest_ch_storages.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/
#include <Functions/FunctionFactory.h>
#include <Parser/MergeTreeRelParser.h>
#include <Parser/RelParsers/MergeTreeRelParser.h>
#include <Processors/Executors/PipelineExecutor.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/SparkMergeTreeMeta.h>
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 @@ -22,8 +22,8 @@
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
#include <Interpreters/Context.h>
#include <Parser/LocalExecutor.h>
#include <Parser/RelParsers/WriteRelParser.h>
#include <Parser/TypeParser.h>
#include <Parser/WriteRelParser.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Processors/Chunk.h>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parser/LocalExecutor.h>
#include <Parser/RelParsers/WriteRelParser.h>
#include <Parser/SubstraitParserUtils.h>
#include <Parser/WriteRelParser.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
Expand Down

0 comments on commit f8c4599

Please sign in to comment.