Merge branch 'master' into youtube-dislikes

This commit is contained in:
Rich Raposa 2023-03-26 16:11:34 -06:00 committed by GitHub
commit 6246aaa168
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 181 additions and 82 deletions

View File

@ -85,8 +85,8 @@ make the matching more natural, all query-level settings related to the query ca
If the query was aborted due to an exception or user cancellation, no entry is written into the query cache. If the query was aborted due to an exception or user cancellation, no entry is written into the query cache.
The size of the query cache, the maximum number of cache entries and the maximum size of cache entries (in bytes and in records) can The size of the query cache in bytes, the maximum number of cache entries and the maximum size of individual cache entries (in bytes and in
be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache). records) can be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache).
To define how long a query must run at least such that its result can be cached, you can use setting To define how long a query must run at least such that its result can be cached, you can use setting
[query_cache_min_query_duration](settings/settings.md#query-cache-min-query-duration). For example, the result of query [query_cache_min_query_duration](settings/settings.md#query-cache-min-query-duration). For example, the result of query

View File

@ -1361,7 +1361,7 @@ If the table does not exist, ClickHouse will create it. If the structure of the
The following settings are available: The following settings are available:
- `size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB). - `max_size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB).
- `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`. - `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`.
- `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB). - `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB).
- `max_entry_rows`: The maximum number of rows `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil). - `max_entry_rows`: The maximum number of rows `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil).
@ -1369,7 +1369,7 @@ The following settings are available:
Changed settings take effect immediately. Changed settings take effect immediately.
:::warning :::warning
Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query cache altogether. Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `max_size` or disable the query cache altogether.
::: :::
**Example** **Example**

View File

@ -7,20 +7,20 @@ sidebar_position: 101
# 什么是列存储数据库? {#what-is-a-columnar-database} # 什么是列存储数据库? {#what-is-a-columnar-database}
列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是影响整行的操作会按比例变得更昂贵。列存储数据库的同义词是面向列的数据库管理系统。ClickHouse就是这样一个典型的例子。 列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是影响整行的操作会按比例变得更昂贵。列存储数据库的同义词是面向列的数据库管理系统。ClickHouse 就是这样一个典型的例子。
列存储数据库的主要优点是: 列存储数据库的主要优点是:
- 查询只使用许多列其中的少数列。 - 查询只使用许多列其中的少数列。
聚合对大量数据的查询。 - 聚合对大量数据的查询。
按列压缩。 - 按列压缩。
下面是构建报表时传统的面向行系统和柱状数据库之间的区别: 下面是构建报表时传统的面向行系统和柱状数据库之间的区别:
**传统行存储** **传统行存储**
!(传统行存储)(https://clickhouse.com/docs/en/images/row-oriented.gif) ![传统行存储](https://clickhouse.com/docs/assets/images/row-oriented-3e6fd5aa48e3075202d242b4799da8fa.gif)
**列存储** **列存储**
!(列存储)(https://clickhouse.com/docs/en/images/column-oriented.gif) ![列存储](https://clickhouse.com/docs/assets/images/column-oriented-d082e49b7743d4ded32c7952bfdb028f.gif)
列存储数据库是分析应用程序的首选因为它允许在一个表中有许多列以防万一但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的因为和数据仓库一样它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。 列存储数据库是分析应用程序的首选因为它允许在一个表中有许多列以防万一但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的因为和数据仓库一样它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse 结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。

View File

@ -1517,7 +1517,7 @@
<!-- Configuration for the query cache --> <!-- Configuration for the query cache -->
<!-- <query_cache> --> <!-- <query_cache> -->
<!-- <size>1073741824</size> --> <!-- <max_size>1073741824</max_size> -->
<!-- <max_entries>1024</max_entries> --> <!-- <max_entries>1024</max_entries> -->
<!-- <max_entry_size>1048576</max_entry_size> --> <!-- <max_entry_size>1048576</max_entry_size> -->
<!-- <max_entry_rows>30000000</max_entry_rows> --> <!-- <max_entry_rows>30000000</max_entry_rows> -->

View File

@ -24,7 +24,6 @@
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/UseSSL.h> #include <IO/UseSSL.h>
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
@ -684,43 +683,76 @@ void QueryFuzzer::fuzzTableName(ASTTableExpression & table)
void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain) void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain)
{ {
/// Fuzz ExplainKind explain.setExplainKind(fuzzExplainKind(explain.getKind()));
bool settings_have_fuzzed = false;
for (auto & child : explain.children)
{
if (auto * settings_ast = typeid_cast<ASTSetQuery *>(child.get()))
{
fuzzExplainSettings(*settings_ast, explain.getKind());
settings_have_fuzzed = true;
}
/// Fuzzing other child like Explain Query
else
{
fuzz(child);
}
}
if (!settings_have_fuzzed)
{
auto settings_ast = std::make_shared<ASTSetQuery>();
settings_ast->is_standalone = false;
fuzzExplainSettings(*settings_ast, explain.getKind());
explain.setSettings(settings_ast);
}
}
ASTExplainQuery::ExplainKind QueryFuzzer::fuzzExplainKind(ASTExplainQuery::ExplainKind kind)
{
if (fuzz_rand() % 20 == 0) if (fuzz_rand() % 20 == 0)
{ {
/// Do not modify ExplainKind return kind;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::ParsedAST); return ASTExplainQuery::ExplainKind::ParsedAST;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::AnalyzedSyntax); return ASTExplainQuery::ExplainKind::AnalyzedSyntax;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryTree); return ASTExplainQuery::ExplainKind::QueryTree;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPlan); return ASTExplainQuery::ExplainKind::QueryPlan;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryPipeline); return ASTExplainQuery::ExplainKind::QueryPipeline;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::QueryEstimates); return ASTExplainQuery::ExplainKind::QueryEstimates;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::TableOverride); return ASTExplainQuery::ExplainKind::TableOverride;
} }
else if (fuzz_rand() % 11 == 0) else if (fuzz_rand() % 11 == 0)
{ {
explain.setExplainKind(ASTExplainQuery::ExplainKind::CurrentTransaction); return ASTExplainQuery::ExplainKind::CurrentTransaction;
} }
return kind;
}
void QueryFuzzer::fuzzExplainSettings(ASTSetQuery & settings_ast, ASTExplainQuery::ExplainKind kind)
{
auto & changes = settings_ast.changes;
static const std::unordered_map<ASTExplainQuery::ExplainKind, std::vector<String>> settings_by_kind static const std::unordered_map<ASTExplainQuery::ExplainKind, std::vector<String>> settings_by_kind
= {{ASTExplainQuery::ExplainKind::ParsedAST, {"graph", "optimize"}}, = {{ASTExplainQuery::ExplainKind::ParsedAST, {"graph", "optimize"}},
@ -732,44 +764,17 @@ void QueryFuzzer::fuzzExplainQuery(ASTExplainQuery & explain)
{ASTExplainQuery::ExplainKind::TableOverride, {}}, {ASTExplainQuery::ExplainKind::TableOverride, {}},
{ASTExplainQuery::ExplainKind::CurrentTransaction, {}}}; {ASTExplainQuery::ExplainKind::CurrentTransaction, {}}};
const auto & settings = settings_by_kind.at(explain.getKind()); const auto & settings = settings_by_kind.at(kind);
bool settings_have_fuzzed = false;
for (auto & child : explain.children)
{
if (auto * settings_ast = typeid_cast<ASTSetQuery *>(child.get()))
{
fuzzExplainSettings(*settings_ast, settings);
settings_have_fuzzed = true;
}
/// Fuzz other child like Explain Query
else
{
fuzz(child);
}
}
if (!settings_have_fuzzed && !settings.empty())
{
auto settings_ast = std::make_shared<ASTSetQuery>();
fuzzExplainSettings(*settings_ast, settings);
explain.setSettings(settings_ast);
}
}
void QueryFuzzer::fuzzExplainSettings(ASTSetQuery & settings, const std::vector<String> & names)
{
auto & changes = settings.changes;
if (fuzz_rand() % 50 == 0 && !changes.empty()) if (fuzz_rand() % 50 == 0 && !changes.empty())
{ {
changes.erase(changes.begin() + fuzz_rand() % changes.size()); changes.erase(changes.begin() + fuzz_rand() % changes.size());
} }
for (const auto & name : names) for (const auto & setting : settings)
{ {
if (fuzz_rand() % 5 == 0) if (fuzz_rand() % 5 == 0)
{ {
changes.emplace_back(name, true); changes.emplace_back(setting, true);
} }
} }
} }
@ -910,6 +915,20 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
if (auto * with_union = typeid_cast<ASTSelectWithUnionQuery *>(ast.get())) if (auto * with_union = typeid_cast<ASTSelectWithUnionQuery *>(ast.get()))
{ {
fuzz(with_union->list_of_selects); fuzz(with_union->list_of_selects);
/// Fuzzing SELECT query to EXPLAIN query randomly.
/// And we only fuzzing the root query into an EXPLAIN query, not fuzzing subquery
if (fuzz_rand() % 20 == 0 && current_ast_depth <= 1)
{
auto explain = std::make_shared<ASTExplainQuery>(fuzzExplainKind());
auto settings_ast = std::make_shared<ASTSetQuery>();
settings_ast->is_standalone = false;
fuzzExplainSettings(*settings_ast, explain->getKind());
explain->setSettings(settings_ast);
explain->setExplainedQuery(ast);
ast = explain;
}
} }
else if (auto * with_intersect_except = typeid_cast<ASTSelectIntersectExceptQuery *>(ast.get())) else if (auto * with_intersect_except = typeid_cast<ASTSelectIntersectExceptQuery *>(ast.get()))
{ {
@ -1086,7 +1105,17 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
} }
else if (auto * explain_query = typeid_cast<ASTExplainQuery *>(ast.get())) else if (auto * explain_query = typeid_cast<ASTExplainQuery *>(ast.get()))
{ {
fuzzExplainQuery(*explain_query); /// Fuzzing EXPLAIN query to SELECT query randomly
if (fuzz_rand() % 20 == 0 && explain_query->getExplainedQuery()->getQueryKind() == IAST::QueryKind::Select)
{
auto select_query = explain_query->getExplainedQuery()->clone();
fuzz(select_query);
ast = select_query;
}
else
{
fuzzExplainQuery(*explain_query);
}
} }
else else
{ {

View File

@ -7,10 +7,11 @@
#include <pcg-random/pcg_random.hpp> #include <pcg-random/pcg_random.hpp>
#include <Core/Field.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/IAST.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
#include "Parsers/IAST_fwd.h" #include "Parsers/IAST_fwd.h"
#include <Core/Field.h>
#include <Parsers/IAST.h>
namespace DB namespace DB
@ -22,7 +23,6 @@ class ASTCreateQuery;
class ASTInsertQuery; class ASTInsertQuery;
class ASTColumnDeclaration; class ASTColumnDeclaration;
class ASTDropQuery; class ASTDropQuery;
class ASTExplainQuery;
class ASTSetQuery; class ASTSetQuery;
struct ASTTableExpression; struct ASTTableExpression;
struct ASTWindowDefinition; struct ASTWindowDefinition;
@ -89,7 +89,8 @@ struct QueryFuzzer
void fuzzWindowFrame(ASTWindowDefinition & def); void fuzzWindowFrame(ASTWindowDefinition & def);
void fuzzCreateQuery(ASTCreateQuery & create); void fuzzCreateQuery(ASTCreateQuery & create);
void fuzzExplainQuery(ASTExplainQuery & explain); void fuzzExplainQuery(ASTExplainQuery & explain);
void fuzzExplainSettings(ASTSetQuery & settings, const std::vector<String> & names); ASTExplainQuery::ExplainKind fuzzExplainKind(ASTExplainQuery::ExplainKind kind = ASTExplainQuery::ExplainKind::QueryPipeline);
void fuzzExplainSettings(ASTSetQuery & settings_ast, ASTExplainQuery::ExplainKind kind);
void fuzzColumnDeclaration(ASTColumnDeclaration & column); void fuzzColumnDeclaration(ASTColumnDeclaration & column);
void fuzzTableName(ASTTableExpression & table); void fuzzTableName(ASTTableExpression & table);
void fuzz(ASTs & asts); void fuzz(ASTs & asts);

View File

@ -86,7 +86,9 @@ struct ColumnToPointsConverter
} }
}; };
/**
* Class which converts Column with type Array(Tuple(Float64, Float64)) to a vector of boost ring type.
*/
template <typename Point> template <typename Point>
struct ColumnToRingsConverter struct ColumnToRingsConverter
{ {
@ -106,7 +108,9 @@ struct ColumnToRingsConverter
} }
}; };
/**
* Class which converts Column with type Array(Array(Tuple(Float64, Float64))) to a vector of boost polygon type.
*/
template <typename Point> template <typename Point>
struct ColumnToPolygonsConverter struct ColumnToPolygonsConverter
{ {
@ -120,6 +124,12 @@ struct ColumnToPolygonsConverter
for (size_t iter = 0; iter < offsets.size(); ++iter) for (size_t iter = 0; iter < offsets.size(); ++iter)
{ {
const auto current_array_size = offsets[iter] - prev_offset; const auto current_array_size = offsets[iter] - prev_offset;
if (current_array_size == 0)
{
answer.emplace_back();
continue;
}
answer[iter].outer() = std::move(all_rings[prev_offset]); answer[iter].outer() = std::move(all_rings[prev_offset]);
answer[iter].inners().reserve(current_array_size); answer[iter].inners().reserve(current_array_size);
for (size_t inner_holes = prev_offset + 1; inner_holes < offsets[iter]; ++inner_holes) for (size_t inner_holes = prev_offset + 1; inner_holes < offsets[iter]; ++inner_holes)
@ -131,7 +141,9 @@ struct ColumnToPolygonsConverter
} }
}; };
/**
* Class which converts Column with type Array(Array(Array(Tuple(Float64, Float64)))) to a vector of boost multi_polygon type.
*/
template <typename Point> template <typename Point>
struct ColumnToMultiPolygonsConverter struct ColumnToMultiPolygonsConverter
{ {
@ -143,7 +155,7 @@ struct ColumnToMultiPolygonsConverter
auto all_polygons = ColumnToPolygonsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr()); auto all_polygons = ColumnToPolygonsConverter<Point>::convert(typeid_cast<const ColumnArray &>(*col).getDataPtr());
for (size_t iter = 0; iter < offsets.size(); ++iter) for (size_t iter = 0; iter < offsets.size() && iter < all_polygons.size(); ++iter)
{ {
for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter) for (size_t polygon_iter = prev_offset; polygon_iter < offsets[iter]; ++polygon_iter)
answer[iter].emplace_back(std::move(all_polygons[polygon_iter])); answer[iter].emplace_back(std::move(all_polygons[polygon_iter]));

View File

@ -20,7 +20,7 @@ public:
String getName() const override { return Name::name; } String getName() const override { return Name::name; }
bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override
{ {

View File

@ -316,7 +316,7 @@ void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & c
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
size_t max_size_in_bytes = config.getUInt64("query_cache.size", 1_GiB); size_t max_size_in_bytes = config.getUInt64("query_cache.max_size", 1_GiB);
cache.setMaxSize(max_size_in_bytes); cache.setMaxSize(max_size_in_bytes);
size_t max_entries = config.getUInt64("query_cache.max_entries", 1024); size_t max_entries = config.getUInt64("query_cache.max_entries", 1024);

View File

@ -94,7 +94,8 @@ std::pair<std::vector<Values>, std::vector<RangesInDataParts>> split(RangesInDat
parts_ranges_queue.push( parts_ranges_queue.push(
{index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeStart}); {index_access->getValue(part_idx, range.begin), {range, part_idx}, PartsRangesIterator::EventType::RangeStart});
const auto & index_granularity = parts[part_idx].data_part->index_granularity; const auto & index_granularity = parts[part_idx].data_part->index_granularity;
if (index_granularity.hasFinalMark() && range.end + 1 == index_granularity.getMarksCount()) const bool value_is_defined_at_end_mark = range.end < index_granularity.getMarksCount();
if (value_is_defined_at_end_mark)
parts_ranges_queue.push( parts_ranges_queue.push(
{index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnd}); {index_access->getValue(part_idx, range.end), {range, part_idx}, PartsRangesIterator::EventType::RangeEnd});
} }

View File

@ -117,30 +117,47 @@ void PartMetadataManagerWithCache::updateAll(bool include_projection)
String value; String value;
String read_value; String read_value;
for (const auto & file_name : file_names)
/// This is used to remove the keys in case of any exception while caching other keys
Strings keys_added_to_cache;
keys_added_to_cache.reserve(file_names.size());
try
{ {
String file_path = fs::path(part->getDataPartStorage().getRelativePath()) / file_name; for (const auto & file_name : file_names)
if (!part->getDataPartStorage().exists(file_name))
continue;
auto in = part->getDataPartStorage().readFile(file_name, {}, std::nullopt, std::nullopt);
readStringUntilEOF(value, *in);
String key = getKeyFromFilePath(file_path);
auto status = cache->put(key, value);
if (!status.ok())
{ {
status = cache->get(key, read_value); String file_path = fs::path(part->getDataPartStorage().getRelativePath()) / file_name;
if (status.IsNotFound() || read_value == value) if (!part->getDataPartStorage().exists(file_name))
continue; continue;
auto in = part->getDataPartStorage().readFile(file_name, {}, std::nullopt, std::nullopt);
readStringUntilEOF(value, *in);
throw Exception( String key = getKeyFromFilePath(file_path);
ErrorCodes::LOGICAL_ERROR, auto status = cache->put(key, value);
"updateAll failed include_projection:{} status:{}, file_path:{}", if (!status.ok())
include_projection, {
status.ToString(), status = cache->get(key, read_value);
file_path); if (status.IsNotFound() || read_value == value)
continue;
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"updateAll failed include_projection:{} status:{}, file_path:{}",
include_projection,
status.ToString(),
file_path);
}
keys_added_to_cache.emplace_back(key);
} }
} }
catch (...)
{
for (const auto & key : keys_added_to_cache)
{
cache->del(key);
}
throw;
}
} }
void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const void PartMetadataManagerWithCache::assertAllDeleted(bool include_projection) const

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
# Tags: no-random-merge-tree-settings
# shellcheck disable=SC2154
unset CLICKHOUSE_LOG_COMMENT
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "CREATE TABLE sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) ENGINE = CollapsingMergeTree(Sign) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) SAMPLE BY intHash32(UserID)"
$CLICKHOUSE_CLIENT -q "INSERT INTO sample_final SELECT number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), if((number % 3) = 1, -1, 1) FROM numbers(1000000)"
query_id="${CLICKHOUSE_DATABASE}_final_excessive_reading_bug_$RANDOM"
$CLICKHOUSE_CLIENT --query_id="$query_id" -q "select * from sample_final FINAL SAMPLE 1/2 OFFSET 1/2 format Null settings max_threads=16"
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "
SELECT ProfileEvents['SelectedRows'] < 1_000_000
FROM system.query_log
WHERE event_date >= yesterday() AND type = 'QueryFinish' AND query_id = {query_id:String} AND current_database = currentDatabase()"

View File

@ -0,0 +1,9 @@
[]
[]
[[(2147483647,0),(10.0001,65535),(1,255),(1023,2147483646)]] [[[(2147483647,0),(10.0001,65535),(1023,2147483646),(2147483647,0)]]]
[[(2147483647,0),(10.0001,65535),(1,255),(1023,2147483646)]] []
[[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]]
[[[(100.0001,1000.0001),(1000.0001,1.1920928955078125e-7),(20,-20),(20,20),(10,10),(-20,20),(100.0001,1000.0001)]]]
[(9223372036854775807,1.1754943508222875e-38)] [[(1,1.0001)]] \N []
[(9223372036854775807,1.1754943508222875e-38)] [] \N []

View File

@ -0,0 +1,5 @@
SELECT polygonsSymDifferenceCartesian([[[(1., 1.)]] AS x], [x]) GROUP BY x WITH ROLLUP;
SELECT [[(2147483647, 0.), (10.0001, 65535), (1, 255), (1023, 2147483646)]], polygonsSymDifferenceCartesian([[[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]]], [[[(1000.0001, 10.0001)]]]) GROUP BY [[(2147483647, 0.), (10.0001, 65535), (1023, 2147483646)]] WITH ROLLUP;
SELECT polygonsSymDifferenceCartesian([[[(100.0001, 1000.0001), (-20., 20.), (10., 10.), (20., 20.), (20., -20.), (1000.0001, 1.1920928955078125e-7)]],[[(0.0001, 100000000000000000000.)]] AS x],[x]) GROUP BY x WITH ROLLUP;
SELECT [(9223372036854775807, 1.1754943508222875e-38)], x, NULL, polygonsSymDifferenceCartesian([[[(1.1754943508222875e-38, 1.1920928955078125e-7), (0.5, 0.5)]], [[(1.1754943508222875e-38, 1.1920928955078125e-7), (1.1754943508222875e-38, 1.1920928955078125e-7)], [(0., 1.0001)]], [[(1., 1.0001)]] AS x], [[[(3.4028234663852886e38, 0.9999)]]]) GROUP BY GROUPING SETS ((x)) WITH TOTALS