Skip to content

Commit

Permalink
[GLUTEN-6467][CH] Minor Fix Build (#6468)
Browse files Browse the repository at this point in the history
[GLUTEN-6467][CH] Minor Fix Build
  • Loading branch information
baibaichen authored Jul 16, 2024
1 parent d2182b3 commit 306791d
Show file tree
Hide file tree
Showing 18 changed files with 26 additions and 21 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,11 @@
#include <IO/ReadHelpers.h>
#include <Interpreters/BloomFilter.h>

namespace DB::ErrorCodes
{
extern const int BAD_ARGUMENTS;
}

namespace local_engine
{
using namespace DB;
Expand Down
1 change: 1 addition & 0 deletions cpp-ch/local-engine/Common/CHUtil.h
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/NamesAndTypes.h>
#include <Core/Settings.h>
#include <Functions/CastOverloadResolver.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/Context.h>
Expand Down
1 change: 1 addition & 0 deletions cpp-ch/local-engine/Common/GlutenSignalHandler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
#include <cstring>
#include <vector>
#include <Core/Settings.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,12 @@
* limitations under the License.
*/
#include "config.h"
#include <Core/Settings.h>
#include <Disks/ObjectStorages/ObjectStorageFactory.h>
#if USE_AWS_S3
#include <Disks/ObjectStorages/S3/DiskS3Utils.h>
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
#include <Disks/ObjectStorages/S3/diskSettings.h>
#include <Disks/ObjectStorages/S3/DiskS3Utils.h>
#endif

#if USE_HDFS
Expand Down
5 changes: 1 addition & 4 deletions cpp-ch/local-engine/Functions/FunctionsBloomFilter.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
#include <memory>
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionGroupBloomFilter.h>
#include <AggregateFunctions/IAggregateFunction_fwd.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/ColumnConst.h>
Expand All @@ -40,15 +41,11 @@
#include <Common/typeid_cast.h>


#include <AggregateFunctions/AggregateFunctionGroupBloomFilter.h>


namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
}

Expand Down
1 change: 1 addition & 0 deletions cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
#pragma once
#include <shared_mutex>
#include <Core/Joins.h>
#include <Interpreters/JoinUtils.h>
#include <Storages/StorageInMemoryMetadata.h>

Expand Down
1 change: 1 addition & 0 deletions cpp-ch/local-engine/Parser/JoinRelParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

#include <memory>
#include <unordered_set>
#include <Core/Joins.h>
#include <Parser/RelParser.h>
#include <substrait/algebra.pb.h>

Expand Down
5 changes: 3 additions & 2 deletions cpp-ch/local-engine/Shuffle/PartitionWriter.h
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,14 @@
#include <memory>
#include <vector>
#include <Core/Block.h>
#include <Core/Settings.h>
#include <IO/WriteBuffer.h>
#include <Interpreters/TemporaryDataOnDisk.h>
#include <Parser/SerializedPlanParser.h>
#include <Shuffle/CachedShuffleWriter.h>
#include <Shuffle/ShuffleSplitter.h>
#include <jni/CelebornClient.h>
#include <Parser/SerializedPlanParser.h>

#include "CachedShuffleWriter.h"

namespace DB
{
Expand Down
4 changes: 2 additions & 2 deletions cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,9 @@
*/
#include "CustomStorageMergeTree.h"


#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/MergeTree/checkDataPart.h>

namespace DB
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
#include "MetaDataHelper.h"

#include <filesystem>

#include <Core/Settings.h>
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <Parser/MergeTreeRelParser.h>
#include <Storages/Mergetree/MergeSparkMergeTreeTask.h>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@
* limitations under the License.
*/
#include "ExcelTextFormatFile.h"
#include <Common/CHUtil.h>

#include <memory>
#include <string>

Expand All @@ -31,6 +29,7 @@
#include <Storages/Serializations/ExcelDecimalSerialization.h>
#include <Storages/Serializations/ExcelSerialization.h>
#include <Storages/Serializations/ExcelStringReader.h>
#include <Common/CHUtil.h>

namespace DB
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,6 @@
*/
#pragma once


#include <memory>
#include <Columns/IColumn.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/ReadBuffer.h>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
#include <memory>
#include <shared_mutex>
#include <thread>
#include <Core/Settings.h>
#include <Disks/IO/AsynchronousBoundedReadBuffer.h>
#include <Disks/IO/ReadBufferFromAzureBlobStorage.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include <Interpreters/Context_fwd.h>
#include <boost/core/noncopyable.hpp>
#include <substrait/plan.pb.h>
#include <Poco/URI.h>

namespace local_engine
{
Expand Down
6 changes: 1 addition & 5 deletions cpp-ch/local-engine/tests/gtest_ch_join.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,11 @@
#include <Common/DebugUtils.h>
#include <Common/MergeTreeTool.h>

#include <Core/Settings.h>
#include <Interpreters/HashJoin/HashJoin.h>
#include <Interpreters/TableJoin.h>
#include <substrait/plan.pb.h>


using namespace DB;
using namespace local_engine;

Expand Down Expand Up @@ -95,17 +95,13 @@ TEST(TestJoin, simple)
ASTPtr rkey = std::make_shared<ASTIdentifier>("colD");
join->addOnKeys(lkey, rkey, false);
for (const auto & column : join->columnsFromJoinedTable())
{
join->addJoinedColumn(column);
}

auto left_keys = left.getNamesAndTypesList();
join->addJoinedColumnsAndCorrectTypes(left_keys, true);
std::cerr << "after join:\n";
for (const auto & key : left_keys)
{
std::cerr << key.dump() << std::endl;
}
ActionsDAGPtr left_convert_actions = nullptr;
ActionsDAGPtr right_convert_actions = nullptr;
std::tie(left_convert_actions, right_convert_actions)
Expand Down
2 changes: 1 addition & 1 deletion cpp-ch/local-engine/tests/gtest_clickhouse_54881.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
*/
#include <gluten_test_util.h>
#include <incbin.h>

#include <Core/Settings.h>
#include <Parser/SerializedPlanParser.h>
#include <gtest/gtest.h>
#include <Common/DebugUtils.h>
Expand Down
1 change: 1 addition & 0 deletions cpp-ch/local-engine/tests/gtest_parquet_columnindex.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
#include <Parser/SerializedPlanParser.h>
#include <Parsers/ExpressionListParsers.h>

#include <Core/Settings.h>
#include <Processors/Formats/Impl/ArrowBufferedStreams.h>
#include <Storages/Parquet/ArrowUtils.h>
#include <Storages/Parquet/ColumnIndexFilter.h>
Expand Down
3 changes: 2 additions & 1 deletion cpp-ch/local-engine/tests/gtest_parser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
#include <gluten_test_util.h>
#include <incbin.h>
#include <Core/Settings.h>
#include <Parser/SerializedPlanParser.h>
#include <gtest/gtest.h>

Expand Down Expand Up @@ -138,7 +139,7 @@ TEST(LocalExecutor, StorageFileSink)
metadata.setColumns(ColumnsDescription::fromNamesAndTypes({{"name", STRING()}, {"value", UINT()}}));
StorageMetadataPtr metadata_ptr = std::make_shared<StorageInMemoryMetadata>(metadata);

/*
/*
auto sink = createFilelinkSink(
metadata_ptr,
"test_table",
Expand Down

0 comments on commit 306791d

Please sign in to comment.