mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
fix joins and rename ENABLE_MONGODB -> USE_MONGODB
This commit is contained in:
parent
49befa714f
commit
e5119ce61f
@ -4,7 +4,7 @@ add_subdirectory (Data/ODBC)
|
||||
add_subdirectory (Foundation)
|
||||
add_subdirectory (JSON)
|
||||
|
||||
if (ENABLE_MONGODB)
|
||||
if (USE_MONGODB)
|
||||
add_subdirectory(MongoDB)
|
||||
endif()
|
||||
|
||||
|
4
contrib/CMakeLists.txt
vendored
4
contrib/CMakeLists.txt
vendored
@ -155,8 +155,8 @@ add_contrib (datasketches-cpp-cmake datasketches-cpp)
|
||||
add_contrib (incbin-cmake incbin)
|
||||
add_contrib (sqids-cpp-cmake sqids-cpp)
|
||||
|
||||
option(ENABLE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
if (ENABLE_MONGODB)
|
||||
option(USE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
if (USE_MONGODB)
|
||||
add_contrib (mongo-c-driver-cmake mongo-c-driver) # requires: zlib
|
||||
add_contrib (mongo-cxx-driver-cmake mongo-cxx-driver) # requires: libmongoc, libbson
|
||||
endif()
|
||||
|
@ -1,5 +1,5 @@
|
||||
option(ENABLE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
if (NOT ENABLE_MONGODB)
|
||||
option(USE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
if (NOT USE_MONGODB)
|
||||
message(STATUS "Not using libmongoc and libbson")
|
||||
return()
|
||||
endif()
|
||||
|
@ -1,6 +1,6 @@
|
||||
option(ENABLE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
option(USE_MONGODB "Enable MongoDB support" ${ENABLE_LIBRARIES})
|
||||
|
||||
if (NOT ENABLE_MONGODB)
|
||||
if (NOT USE_MONGODB)
|
||||
message(STATUS "Not using mongocxx and bsoncxx")
|
||||
return()
|
||||
endif()
|
||||
|
@ -486,6 +486,8 @@ try
|
||||
Poco::ErrorHandler::set(&error_handler);
|
||||
}
|
||||
|
||||
processConfig();
|
||||
|
||||
registerInterpreters();
|
||||
/// Don't initialize DateLUT
|
||||
registerFunctions();
|
||||
@ -497,8 +499,6 @@ try
|
||||
registerDisks(/* global_skip_access_check= */ true);
|
||||
registerFormats();
|
||||
|
||||
processConfig();
|
||||
|
||||
SCOPE_EXIT({ cleanup(); });
|
||||
|
||||
initTTYBuffer(toProgressOption(getClientConfiguration().getString("progress", "default")));
|
||||
|
@ -412,7 +412,7 @@ dbms_target_link_libraries (
|
||||
Poco::Redis
|
||||
)
|
||||
|
||||
if (ENABLE_MONGODB)
|
||||
if (USE_MONGODB)
|
||||
dbms_target_link_libraries (PUBLIC Poco::MongoDB)
|
||||
endif()
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Common/Base64.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
|
||||
|
@ -64,7 +64,7 @@
|
||||
#cmakedefine01 USE_LIBARCHIVE
|
||||
#cmakedefine01 USE_POCKETFFT
|
||||
#cmakedefine01 USE_PROMETHEUS_PROTOBUFS
|
||||
#cmakedefine01 ENABLE_MONGODB
|
||||
#cmakedefine01 USE_MONGODB
|
||||
#cmakedefine01 USE_NUMACTL
|
||||
|
||||
/// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO.
|
||||
|
@ -40,7 +40,7 @@ target_link_libraries(clickhouse_dictionaries
|
||||
Poco::Redis
|
||||
)
|
||||
|
||||
if (ENABLE_MONGODB)
|
||||
if (USE_MONGODB)
|
||||
target_link_libraries(clickhouse_dictionaries PRIVATE Poco::MongoDB)
|
||||
endif()
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include "config.h"
|
||||
|
||||
#include "DictionarySourceFactory.h"
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "MongoDBDictionarySource.h"
|
||||
#include "DictionaryStructure.h"
|
||||
|
||||
@ -21,7 +21,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int LOGICAL_ERROR;
|
||||
#else
|
||||
@ -31,7 +31,7 @@ namespace ErrorCodes
|
||||
|
||||
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
{
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
auto create_dictionary_source = [](
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
@ -109,7 +109,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
|
||||
factory.registerSource("mongodb", create_dictionary_source);
|
||||
}
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
static const UInt64 max_block_size = 8192;
|
||||
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "DictionaryStructure.h"
|
||||
#include "IDictionarySource.h"
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include "config.h"
|
||||
|
||||
#include "DictionarySourceFactory.h"
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "MongoDBPocoLegacyDictionarySource.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "registerDictionaries.h"
|
||||
@ -14,7 +14,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int MONGODB_CANNOT_AUTHENTICATE;
|
||||
@ -23,14 +23,14 @@ extern const int SUPPORT_IS_DISABLED;
|
||||
#endif
|
||||
}
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
static const std::unordered_set<std::string_view> dictionary_allowed_keys = {
|
||||
"host", "port", "user", "password", "db", "database", "uri", "collection", "name", "method", "options"};
|
||||
#endif
|
||||
|
||||
void registerDictionarySourceMongoDBPocoLegacy(DictionarySourceFactory & factory)
|
||||
{
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
auto create_mongo_db_dictionary = [](
|
||||
const DictionaryStructure & dict_struct,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
@ -92,7 +92,7 @@ void registerDictionarySourceMongoDBPocoLegacy(DictionarySourceFactory & factory
|
||||
|
||||
}
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Poco/MongoDB/Array.h>
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Processors/Sources/MongoDBPocoLegacySource.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "MongoDBPocoLegacySource.h"
|
||||
|
||||
#include <string>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Poco/MongoDB/Element.h>
|
||||
#include <Poco/MongoDB/Array.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "MongoDBSource.h"
|
||||
|
||||
#include <vector>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Processors/ISource.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <memory>
|
||||
|
||||
#include <Analyzer/ColumnNode.h>
|
||||
@ -182,7 +182,11 @@ std::string mongoFuncName(const std::string & func)
|
||||
}
|
||||
|
||||
template <typename OnError>
|
||||
std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const ContextPtr & context, const FunctionNode * func, OnError on_error)
|
||||
std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(
|
||||
const ContextPtr & context,
|
||||
const FunctionNode * func,
|
||||
const JoinNode * join_node,
|
||||
OnError on_error)
|
||||
{
|
||||
if (func->getArguments().getNodes().empty())
|
||||
return {};
|
||||
@ -197,6 +201,8 @@ std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const
|
||||
// Skip columns from other tables in JOIN queries.
|
||||
if (table->getStorage()->getStorageID() != this->getStorageID())
|
||||
return {};
|
||||
if (join_node && column->getColumnSource() != join_node->getLeftTableExpression())
|
||||
return {};
|
||||
|
||||
// Only these function can have exactly one argument and be passed to MongoDB.
|
||||
if (func->getFunctionName() == "isNull")
|
||||
@ -236,7 +242,7 @@ std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const
|
||||
}
|
||||
|
||||
if (const auto & func_value = value->as<FunctionNode>())
|
||||
if (const auto & res_value = visitWhereFunction(context, func_value, on_error); res_value.has_value())
|
||||
if (const auto & res_value = visitWhereFunction(context, func_value, join_node, on_error); res_value.has_value())
|
||||
return make_document(kvp(column->getColumnName(), make_document(kvp(func_name, *res_value))));
|
||||
}
|
||||
}
|
||||
@ -246,7 +252,7 @@ std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const
|
||||
for (const auto & elem : func->getArguments().getNodes())
|
||||
{
|
||||
if (const auto & elem_func = elem->as<FunctionNode>())
|
||||
if (const auto & res_value = visitWhereFunction(context, elem_func, on_error); res_value.has_value())
|
||||
if (const auto & res_value = visitWhereFunction(context, elem_func, join_node, on_error); res_value.has_value())
|
||||
arr.append(*res_value);
|
||||
}
|
||||
if (!arr.view().empty())
|
||||
@ -308,8 +314,8 @@ bsoncxx::document::value StorageMongoDB::buildMongoDBQuery(const ContextPtr & co
|
||||
if (throw_on_error)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Only simple queries are supported, failed to convert expression '{}' to MongoDB query. "
|
||||
"You can disable this restriction with 'SET mongodb_throw_on_unsupported_query=0', to read the full table and process on CLickHouse side (this may cause poor performance)", node->formatASTForErrorMessage());
|
||||
LOG_WARNING(log, "Failed to build MongoDB sort for '{}'", node ? node->formatASTForErrorMessage() : "<unknown>");
|
||||
"You can disable this restriction with 'SET mongodb_throw_on_unsupported_query=0', to read the full table and process on ClickHouse side (this may cause poor performance)", node->formatASTForErrorMessage());
|
||||
LOG_WARNING(log, "Failed to build MongoDB query for '{}'", node ? node->formatASTForErrorMessage() : "<unknown>");
|
||||
};
|
||||
|
||||
|
||||
@ -365,14 +371,14 @@ bsoncxx::document::value StorageMongoDB::buildMongoDBQuery(const ContextPtr & co
|
||||
{
|
||||
std::optional<bsoncxx::document::value> filter{};
|
||||
if (const auto & func = query_tree.getWhere()->as<FunctionNode>())
|
||||
filter = visitWhereFunction(context, func, on_error);
|
||||
filter = visitWhereFunction(context, func, join_node, on_error);
|
||||
|
||||
else if (const auto & const_expr = query_tree.getWhere()->as<ConstantNode>())
|
||||
{
|
||||
if (const_expr->hasSourceExpression())
|
||||
{
|
||||
if (const auto & func_expr = const_expr->getSourceExpression()->as<FunctionNode>())
|
||||
filter = visitWhereFunction(context, func_expr, on_error);
|
||||
filter = visitWhereFunction(context, func_expr, join_node, on_error);
|
||||
}
|
||||
}
|
||||
|
||||
@ -381,7 +387,6 @@ bsoncxx::document::value StorageMongoDB::buildMongoDBQuery(const ContextPtr & co
|
||||
LOG_DEBUG(log, "MongoDB query has built: '{}'.", bsoncxx::to_json(*filter));
|
||||
return std::move(*filter);
|
||||
}
|
||||
on_error(query_tree.getWhere().get());
|
||||
}
|
||||
else
|
||||
on_error(join_node);
|
||||
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Analyzer/JoinNode.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
|
||||
@ -61,8 +61,17 @@ public:
|
||||
|
||||
private:
|
||||
template <typename OnError>
|
||||
std::optional<bsoncxx::document::value> visitWhereFunction(const ContextPtr & context, const FunctionNode * func, OnError on_error);
|
||||
bsoncxx::document::value buildMongoDBQuery(const ContextPtr & context, mongocxx::options::find & options, const SelectQueryInfo & query, const Block & sample_block);
|
||||
std::optional<bsoncxx::document::value> visitWhereFunction(
|
||||
const ContextPtr & context,
|
||||
const FunctionNode * func,
|
||||
const JoinNode * join_node,
|
||||
OnError on_error);
|
||||
|
||||
bsoncxx::document::value buildMongoDBQuery(
|
||||
const ContextPtr & context,
|
||||
mongocxx::options::find & options,
|
||||
const SelectQueryInfo & query,
|
||||
const Block & sample_block);
|
||||
|
||||
const MongoDBConfiguration configuration;
|
||||
LoggerPtr log;
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Storages/StorageMongoDBPocoLegacy.h>
|
||||
#include <Storages/StorageMongoDBPocoLegacySocketFactory.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
|
||||
#include <Storages/IStorage.h>
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include "StorageMongoDBPocoLegacySocketFactory.h"
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Poco/MongoDB/Connection.h>
|
||||
|
||||
|
||||
|
@ -61,7 +61,7 @@ void registerStorageJDBC(StorageFactory & factory);
|
||||
void registerStorageMySQL(StorageFactory & factory);
|
||||
#endif
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
void registerStorageMongoDB(StorageFactory & factory);
|
||||
void registerStorageMongoDBPocoLegacy(StorageFactory & factory);
|
||||
#endif
|
||||
@ -165,7 +165,7 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]])
|
||||
registerStorageMySQL(factory);
|
||||
#endif
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
if (use_legacy_mongodb_integration)
|
||||
registerStorageMongoDBPocoLegacy(factory);
|
||||
else
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Storages/StorageMongoDB.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "config.h"
|
||||
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
#include <Storages/StorageMongoDBPocoLegacy.h>
|
||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||
|
||||
|
@ -22,7 +22,7 @@ void registerTableFunctions(bool use_legacy_mongodb_integration [[maybe_unused]]
|
||||
registerTableFunctionValues(factory);
|
||||
registerTableFunctionInput(factory);
|
||||
registerTableFunctionGenerate(factory);
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
if (use_legacy_mongodb_integration)
|
||||
registerTableFunctionMongoDBPocoLegacy(factory);
|
||||
else
|
||||
|
@ -20,7 +20,7 @@ void registerTableFunctionURLCluster(TableFunctionFactory & factory);
|
||||
void registerTableFunctionValues(TableFunctionFactory & factory);
|
||||
void registerTableFunctionInput(TableFunctionFactory & factory);
|
||||
void registerTableFunctionGenerate(TableFunctionFactory & factory);
|
||||
#if ENABLE_MONGODB
|
||||
#if USE_MONGODB
|
||||
void registerTableFunctionMongoDB(TableFunctionFactory & factory);
|
||||
void registerTableFunctionMongoDBPocoLegacy(TableFunctionFactory & factory);
|
||||
#endif
|
||||
|
@ -174,7 +174,7 @@ if (TARGET ch_contrib::prometheus_protobufs)
|
||||
set(USE_PROMETHEUS_PROTOBUFS 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::mongocxx)
|
||||
set(ENABLE_MONGODB 1)
|
||||
set(USE_MONGODB 1)
|
||||
endif()
|
||||
if (TARGET ch_contrib::numactl)
|
||||
set(USE_NUMACTL 1)
|
||||
|
Loading…
Reference in New Issue
Block a user