This commit is contained in:
vdimir 2024-07-31 12:08:53 +00:00
parent 175f0696d5
commit 6ea69fbfb2
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
10 changed files with 221 additions and 189 deletions

View File

@ -203,7 +203,7 @@ TRAP(lgammal)
TRAP(nftw)
TRAP(nl_langinfo)
TRAP(putc_unlocked)
//TRAP(rand) // Used in mongo-c-driver
TRAP(rand)
/** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern
* implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams

View File

@ -10,77 +10,10 @@ MongoDB engine is read-only table engine which allows to read data from remote [
Only MongoDB v3.6+ servers are supported.
**If you're facing troubles, please report the issue, and try to use [the legacy implementation](../../../operations/server-configuration-parameters/settings.md#use_legacy_mongodb_integration).
Keep in mind that it is deprecated, and will be removed in next releases.**
## Types mappings
| MongoDB | ClickHouse |
|--------------------|-----------------------------------------------------------------------|
| bool, int32, int64 | *any numeric type*, String |
| int32 | Int32, String |
| int64 | Int64, String |
| double | Float64, String |
| date | Date, Date32, DateTime, DateTime64, String |
| string | String, UUID |
| document | String(as JSON) |
| array | Array, String(as JSON) |
| oid | String |
| binary | String if in column, base64 encoded string if in an array or document |
| *any other* | String |
If key not found in MongoDB document, default value or null(if the column is nullable) will be inserted.
## Supported clauses
**You can disable all these restriction, see [mongodb_fail_on_query_build_error](../../../operations/settings/settings.md#mongodb_fail_on_query_build_error).**\
*If `allow_experimental_analyzer=0`, ClickHouse will not try to build MongoDB query, sort and limit.*
#### You can use MongoDB table in CTE to perform any clauses, but be aware, that in some cases, performance will be significantly degraded.
For example, you want to query count() with GROUP BY(which is not supported by MongoDB engine):
```sql
SELECT count(), name FROM mongo_table WHERE name IN ('clickhouse', 'mongodb') GROUP BY name;
```
You can set `mongodb_fail_on_query_build_error=0`, but this will cause poor performance, because all data will be read from `mongo_table`
before filtering by `name`. \
So, there is a solution:
```sql
SELECT count(), name
FROM (SELECT name FROM mongo_table WHERE name in ('clickhouse', 'mongodb'))
GROUP BY name;
```
### WHERE
Only constant literals are allowed.
PREWHERE and HAVING are not supported.
#### Note:
It's always better to explicitly set type of literal because Mongo requires strict typed filters.\
For example you want to filter by `Date`:
```sql
SELECT * FROM mongo_table WHERE date = '2024-01-01'
```
This will not work because Mongo will not cast string to `Date`, so you need to cast it manually:
```sql
SELECT * FROM mongo_table WHERE date = '2024-01-01'::Date OR date = toDate('2024-01-01')
```
This applied for `Date`, `Date32`, `DateTime`, `Bool`, `UUID`.
### LIMIT and OFFSET
Only `LIMIT` is supported.
### ORDER BY
Simple expressions only are supported, without any modification like COLLATE, WITH, TO, etc.
### WINDOW
Not supported.
### GROUP BY
Not supported.
### Aggregation functions
Not supported.
:::note
If you're facing troubles, please report the issue, and try to use [the legacy implementation](../../../operations/server-configuration-parameters/settings.md#use_legacy_mongodb_integration).
Keep in mind that it is deprecated, and will be removed in next releases.
:::
## Creating a Table {#creating-a-table}
@ -114,10 +47,10 @@ If you are using the MongoDB Atlas cloud offering:
- connection url can be obtained from 'Atlas SQL' option
- use options: 'connectTimeoutMS=10000&ssl=true&authSource=admin'
```
:::
Also, you can simply pass a URI:
``` sql
ENGINE = MongoDB(uri, collection);
```
@ -128,52 +61,130 @@ ENGINE = MongoDB(uri, collection);
- `collection` — Remote collection name.
## Types mappings
| MongoDB | ClickHouse |
|--------------------|-----------------------------------------------------------------------|
| bool, int32, int64 | *any numeric type*, String |
| int32 | Int32, String |
| int64 | Int64, String |
| double | Float64, String |
| date | Date, Date32, DateTime, DateTime64, String |
| string | String, UUID |
| document | String(as JSON) |
| array | Array, String(as JSON) |
| oid | String |
| binary | String if in column, base64 encoded string if in an array or document |
| *any other* | String |
If key is not found in MongoDB document (for example, column name doesn't match), default value or `NULL` (if the column is nullable) will be inserted.
## Supported clauses
Only queries with simple expressions are supported (for example, `WHERE field = <constant> ORDER BY field2 LIMIT <constant>`).
Such expressions are translated to MongoDB query language and executed on the server side.
You can disable all these restriction, using [mongodb_throw_on_unsupported_query](../../../operations/settings/settings.md#mongodb_throw_on_unsupported_query).
In that case ClickHouse tries to convert query on best effort basis, but it can lead to full table scan and processing on ClickHouse side.
:::note
It's always better to explicitly set type of literal because Mongo requires strict typed filters.\
For example you want to filter by `Date`:
```sql
SELECT * FROM mongo_table WHERE date = '2024-01-01'
```
This will not work because Mongo will not cast string to `Date`, so you need to cast it manually:
```sql
SELECT * FROM mongo_table WHERE date = '2024-01-01'::Date OR date = toDate('2024-01-01')
```
This applied for `Date`, `Date32`, `DateTime`, `Bool`, `UUID`.
:::
## Usage Example {#usage-example}
Assuming MongoDB has [sample_mflix](https://www.mongodb.com/docs/atlas/sample-data/sample-mflix) dataset loaded
Create a table in ClickHouse which allows to read data from MongoDB collection:
``` sql
CREATE TABLE mongo_table
CREATE TABLE sample_mflix_table
(
key UInt64,
data String
) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'testuser', 'password');
```
or
``` sql
ENGINE = MongoDB('mongodb://testuser:password@mongo1:27017/test', 'simple_table');
```
To read from an SSL secured MongoDB server:
``` sql
CREATE TABLE mongo_table_ssl
(
key UInt64,
data String
) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table', 'testuser', 'password', 'ssl=true');
_id String,
title String,
plot String,
genres Array(String),
directors Array(String),
writers Array(String),
released Date,
imdb String,
year String,
) ENGINE = MongoDB('mongodb+srv://<USERNAME>:<PASSWORD>@cluster0.cdojylq.mongodb.net/sample_mflix', 'movies');
```
Query:
``` sql
SELECT COUNT() FROM mongo_table;
SELECT count() FROM sample_mflix_table
```
``` text
┌─count()─┐
│ 4 │
└─────────┘
┌─count()─┐
1. │ 21349
└─────────┘
```
You can also adjust connection timeout:
```SQL
-- JSONExtractString cannot be pushed down to MongoDB
SET mongodb_throw_on_unsupported_query = 0;
``` sql
CREATE TABLE mongo_table
(
key UInt64,
data String
) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table', 'testuser', 'password', 'connectTimeoutMS=100000');
-- Find all 'Back to the Future' sequels with rating > 7.5
SELECT title, plot, genres, directors, released FROM sample_mflix_table
WHERE title IN ('Back to the Future', 'Back to the Future Part II', 'Back to the Future Part III')
AND toFloat32(JSONExtractString(imdb, 'rating')) > 7.5
ORDER BY year
FORMAT Vertical;
```
```text
Row 1:
──────
title: Back to the Future
plot: A young man is accidentally sent 30 years into the past in a time-traveling DeLorean invented by his friend, Dr. Emmett Brown, and must make sure his high-school-age parents unite in order to save his own existence.
genres: ['Adventure','Comedy','Sci-Fi']
directors: ['Robert Zemeckis']
released: 1985-07-03
Row 2:
──────
title: Back to the Future Part II
plot: After visiting 2015, Marty McFly must repeat his visit to 1955 to prevent disastrous changes to 1985... without interfering with his first trip.
genres: ['Action','Adventure','Comedy']
directors: ['Robert Zemeckis']
released: 1989-11-22
```
```SQL
-- Find top 3 movies based on Cormac McCarthy's books
SELECT title, toFloat32(JSONExtractString(imdb, 'rating')) as rating
FROM sample_mflix_table
WHERE arrayExists(x -> x like 'Cormac McCarthy%', writers)
ORDER BY rating DESC
LIMIT 3;
```
```text
┌─title──────────────────┬─rating─┐
1. │ No Country for Old Men │ 8.1 │
2. │ The Sunset Limited │ 7.4 │
3. │ The Road │ 7.3 │
└────────────────────────┴────────┘
```
## Troubleshooting

View File

@ -5609,7 +5609,7 @@ Minimal size of block to compress in CROSS JOIN. Zero value means - disable this
Default value: `1GiB`.
## mongodb_fail_on_query_build_error
## mongodb_throw_on_unsupported_query
If enabled, MongoDB tables will return an error when a MongoDB query can't be built.

View File

@ -1744,7 +1744,7 @@ SOURCE(MONGODB(
Setting fields:
- 'uri' - URI for establish the connection.
- `uri` - URI for establish the connection.
- `collection` Name of the collection.
[More information about the engine](../../engines/table-engines/integrations/mongodb.md)
@ -2065,7 +2065,7 @@ Configuration fields:
| `expression` | [Expression](../../sql-reference/syntax.md#expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No |
| <a name="hierarchical-dict-attr"></a> `hierarchical` | If `true`, the attribute contains the value of a parent key for the current key. See [Hierarchical Dictionaries](#hierarchical-dictionaries).<br/><br/>Default value: `false`. | No |
| `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).<br/>If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.<br/><br/>Default value: `false`. | No |
| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`.
| `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`.
## Hierarchical Dictionaries

View File

@ -103,32 +103,37 @@ static JSONBuilder::ItemPtr BSONElementAsJSON(const T & value)
return std::make_unique<JSONBuilder::JSONString>(DateLUT::instance().timeToString(value.get_date().to_int64() / 1000));
case bsoncxx::type::k_timestamp:
return std::make_unique<JSONBuilder::JSONString>(DateLUT::instance().timeToString(value.get_timestamp().timestamp));
case bsoncxx::type::k_document: {
case bsoncxx::type::k_document:
{
auto doc = std::make_unique<JSONBuilder::JSONMap>();
for (const auto & elem : value.get_document().value)
doc->add(std::string(elem.key()), BSONElementAsJSON(elem));
return doc;
}
case bsoncxx::type::k_array: {
case bsoncxx::type::k_array:
{
auto arr = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & elem : value.get_array().value)
arr->add(BSONElementAsJSON(elem));
return arr;
}
case bsoncxx::type::k_regex: {
case bsoncxx::type::k_regex:
{
auto doc = std::make_unique<JSONBuilder::JSONMap>();
doc->add(std::string(value.get_regex().regex), std::string(value.get_regex().options));
return doc;
}
case bsoncxx::type::k_dbpointer: {
case bsoncxx::type::k_dbpointer:
{
auto doc = std::make_unique<JSONBuilder::JSONMap>();
doc->add(value.get_dbpointer().value.to_string(), std::string(value.get_dbpointer().collection));
return doc;
}
case bsoncxx::type::k_null:
return std::make_unique<JSONBuilder::JSONNull>();
default:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BSON type {} is unserializable.", bsoncxx::to_string(value.type()));
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization BSON type '{}' is not supported", bsoncxx::to_string(value.type()));
}
}
@ -163,7 +168,8 @@ static std::string BSONElementAsString(const T & value, const JSONBuilder::Forma
case bsoncxx::type::k_array:
case bsoncxx::type::k_regex:
case bsoncxx::type::k_dbpointer:
case bsoncxx::type::k_symbol: {
case bsoncxx::type::k_symbol:
{
WriteBufferFromOwnString buf;
auto format_context = JSONBuilder::FormatContext{.out = buf};
BSONElementAsJSON(value)->format(json_format_settings, format_context);

View File

@ -604,7 +604,6 @@
M(723, PARQUET_EXCEPTION) \
M(724, TOO_MANY_TABLES) \
M(725, TOO_MANY_DATABASES) \
M(726, FAILED_TO_BUILD_MONGODB_QUERY) \
\
M(900, DISTRIBUTED_CACHE_ERROR) \
M(901, CANNOT_USE_DISTRIBUTED_CACHE) \

View File

@ -893,7 +893,7 @@ class IColumn;
M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \
M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \
M(UInt64, extract_key_value_pairs_max_pairs_per_row, 1000, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory.", 0) ALIAS(extract_kvp_max_pairs_per_row) \
M(Bool, mongodb_fail_on_query_build_error, true, "If enabled, MongoDB tables will return an error when a MongoDB query can't be built. Not applied for the legacy implementation, or when 'allow_experimental_analyzer=0`.", 0) \
M(Bool, mongodb_throw_on_unsupported_query, true, "If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'.", 0) \
M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \
M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \
\

View File

@ -37,7 +37,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int FAILED_TO_BUILD_MONGODB_QUERY;
extern const int NOT_IMPLEMENTED;
}
using BSONCXXHelper::fieldAsBSONValue;
@ -176,10 +176,11 @@ std::string mongoFuncName(const std::string & func)
if (func == "or")
return "$or";
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "Function '{}' is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.", func);
return "";
}
std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const ContextPtr & context, const FunctionNode * func)
template <typename OnError>
std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const ContextPtr & context, const FunctionNode * func, OnError on_error)
{
if (func->getArguments().getNodes().empty())
return {};
@ -192,7 +193,7 @@ std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const
return {};
// Skip columns from other tables in JOIN queries.
if (table->getStorage()->getStorageID().getFullTableName() != this->getStorageID().getFullTableName())
if (table->getStorage()->getStorageID() != this->getStorageID())
return {};
// Only these function can have exactly one argument and be passed to MongoDB.
@ -206,6 +207,12 @@ std::optional<bsoncxx::document::value> StorageMongoDB::visitWhereFunction(const
return make_document(kvp(column->getColumnName(), make_document(kvp("$nin", make_array(bsoncxx::types::b_null{}, "")))));
auto func_name = mongoFuncName(func->getFunctionName());
if (func_name.empty())
{
on_error(func);
return {};
}
if (func->getArguments().getNodes().size() == 2)
{
const auto & value = func->getArguments().getNodes().at(1);
@ -227,7 +234,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); res_value.has_value())
if (const auto & res_value = visitWhereFunction(context, func_value, on_error); res_value.has_value())
return make_document(kvp(column->getColumnName(), make_document(kvp(func_name, *res_value))));
}
}
@ -237,17 +244,23 @@ 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); res_value.has_value())
if (const auto & res_value = visitWhereFunction(context, elem_func, on_error); res_value.has_value())
arr.append(*res_value);
}
if (!arr.view().empty())
return make_document(kvp(mongoFuncName(func->getFunctionName()), arr));
{
auto func_name = mongoFuncName(func->getFunctionName());
if (func_name.empty())
{
on_error(func);
return {};
}
return make_document(kvp(func_name, arr));
}
}
throw Exception(
ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY,
"Only constant expressions are supported in WHERE section. You can disable this error with 'SET "
"mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
on_error(func);
return {};
}
bsoncxx::document::value StorageMongoDB::buildMongoDBQuery(const ContextPtr & context, mongocxx::options::find & options, const SelectQueryInfo & query, const Block & sample_block)
@ -255,103 +268,105 @@ bsoncxx::document::value StorageMongoDB::buildMongoDBQuery(const ContextPtr & co
document projection{};
for (const auto & column : sample_block)
projection.append(kvp(column.name, 1));
LOG_DEBUG(log, "MongoDB projection has built: '{}'.", bsoncxx::to_json(projection));
LOG_DEBUG(log, "MongoDB projection has built: '{}'", bsoncxx::to_json(projection));
options.projection(projection.extract());
if (!context->getSettingsRef().allow_experimental_analyzer)
return make_document();
auto & query_tree = query.query_tree->as<QueryNode &>();
bool throw_on_error = context->getSettingsRef().mongodb_throw_on_unsupported_query;
if (context->getSettingsRef().mongodb_fail_on_query_build_error)
if (!context->getSettingsRef().allow_experimental_analyzer)
{
if (throw_on_error)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MongoDB storage does not support 'allow_experimental_analyzer = 0' setting");
return make_document();
}
const auto & query_tree = query.query_tree->as<QueryNode &>();
if (throw_on_error)
{
if (query_tree.hasHaving())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "HAVING section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "HAVING section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
if (query_tree.hasGroupBy())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "GROUP BY section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GROUP BY section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
if (query_tree.hasWindow())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "WINDOW section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WINDOW section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
if (query_tree.hasPrewhere())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "PREWHERE section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PREWHERE section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
if (query_tree.hasLimitBy())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "LIMIT BY section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "LIMIT BY section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
if (query_tree.hasOffset())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "OFFSET section is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "OFFSET section is not supported. You can disable this error with 'SET mongodb_throw_on_unsupported_query=0', but this may cause poor performance, and is highly not recommended");
}
auto on_error = [&] (const auto * node)
{
/// Reset limit, because if we omit ORDER BY, it should not be applied
options.limit(0);
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>");
};
if (query_tree.hasLimit())
{
try
{
if (const auto & limit = query_tree.getLimit()->as<ConstantNode>())
options.limit(limit->getValue().safeGet<UInt64>());
else
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "Only simple limit is supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
}
catch (Exception & e)
{
if (context->getSettingsRef().mongodb_fail_on_query_build_error)
throw;
LOG_WARNING(log, "Failed to build MongoDB limit: '{}'.", e.message());
}
if (const auto & limit = query_tree.getLimit()->as<ConstantNode>())
options.limit(limit->getValue().safeGet<UInt64>());
else
on_error(query_tree.getLimit().get());
}
if (query_tree.hasOrderBy())
{
try
document sort{};
for (const auto & child : query_tree.getOrderByNode()->getChildren())
{
document sort{};
for (const auto & child : query_tree.getOrderByNode()->getChildren())
if (const auto * sort_node = child->as<SortNode>())
{
if (const auto & sort_node = child->as<SortNode>())
{
if (sort_node->withFill() || sort_node->hasFillTo() || sort_node->hasFillFrom() || sort_node->hasFillStep())
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "ORDER BY WITH FILL is not supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
if (const auto & column = sort_node->getExpression()->as<ColumnNode>())
sort.append(kvp(column->getColumnName(), sort_node->getSortDirection() == SortDirection::ASCENDING ? 1 : -1));
else
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "Only simple sort is supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
}
if (sort_node->withFill() || sort_node->hasFillTo() || sort_node->hasFillFrom() || sort_node->hasFillStep())
on_error(sort_node);
if (const auto & column = sort_node->getExpression()->as<ColumnNode>())
sort.append(kvp(column->getColumnName(), sort_node->getSortDirection() == SortDirection::ASCENDING ? 1 : -1));
else
throw Exception(ErrorCodes::FAILED_TO_BUILD_MONGODB_QUERY, "Only simple sort is supported. You can disable this error with 'SET mongodb_fail_on_query_build_error=0', but this may cause poor performance, and is highly not recommended.");
on_error(sort_node);
}
LOG_DEBUG(log, "MongoDB sort has built: '{}'.", bsoncxx::to_json(sort));
options.sort(sort.extract());
else
on_error(sort_node);
}
catch (Exception & e)
if (!sort.view().empty())
{
if (context->getSettingsRef().mongodb_fail_on_query_build_error)
throw;
LOG_WARNING(log, "Failed to build MongoDB sort: '{}'.", e.message());
LOG_DEBUG(log, "MongoDB sort has built: '{}'", bsoncxx::to_json(sort));
options.sort(sort.extract());
}
}
if (query_tree.hasWhere())
{
try
{
std::optional<bsoncxx::document::value> filter{};
if (const auto & func = query_tree.getWhere()->as<FunctionNode>())
filter = visitWhereFunction(context, func);
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);
}
}
std::optional<bsoncxx::document::value> filter{};
if (const auto & func = query_tree.getWhere()->as<FunctionNode>())
filter = visitWhereFunction(context, func, on_error);
if (filter.has_value())
else if (const auto & const_expr = query_tree.getWhere()->as<ConstantNode>())
{
if (const_expr->hasSourceExpression())
{
LOG_DEBUG(log, "MongoDB query has built: '{}'.", bsoncxx::to_json(*filter));
return std::move(*filter);
if (const auto & func_expr = const_expr->getSourceExpression()->as<FunctionNode>())
filter = visitWhereFunction(context, func_expr, on_error);
}
}
catch (Exception & e)
if (filter.has_value())
{
if (context->getSettingsRef().mongodb_fail_on_query_build_error)
throw;
LOG_WARNING(log, "Failed to build MongoDB query: '{}'.", e.message());
LOG_DEBUG(log, "MongoDB query has built: '{}'.", bsoncxx::to_json(*filter));
return std::move(*filter);
}
else
{
on_error(query_tree.getWhere().get());
}
}

View File

@ -60,7 +60,8 @@ public:
size_t num_streams) override;
private:
std::optional<bsoncxx::document::value> visitWhereFunction(const ContextPtr & context, const FunctionNode * func);
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);
const MongoDBConfiguration configuration;

View File

@ -173,7 +173,7 @@ endif()
if (TARGET ch_contrib::prometheus_protobufs)
set(USE_PROMETHEUS_PROTOBUFS 1)
endif()
if (ch_contrib::mongo-cxx-driver)
if (TARGET ch_contrib::mongocxx)
set(USE_MONGODB 1)
endif()
if (TARGET ch_contrib::numactl)